From c35524dae2d430d5ceb39b07e9124fac5d02648b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 10 Oct 2023 12:57:23 +0300 Subject: [PATCH 01/37] [fix][test] Fix flaky AdminApiMaxUnackedMessagesTest.testMaxUnackedMessagesPerConsumerPriority (#21335) --- .../pulsar/broker/admin/AdminApiMaxUnackedMessagesTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessagesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessagesTest.java index 4fb268fc9713f..9b95e41bb64e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessagesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMaxUnackedMessagesTest.java @@ -197,7 +197,7 @@ public void testMaxUnackedMessagesPerConsumerPriority() throws Exception { private List consumeMsg(Consumer consumer, int msgNum) throws Exception { List list = new ArrayList<>(); for (int i = 0; i Date: Tue, 10 Oct 2023 23:35:36 +0900 Subject: [PATCH 02/37] [feat][cli] Add command line option for configuring the memory limit (#20663) Signed-off-by: tison Co-authored-by: tison --- .../client/cli/PulsarClientToolTest.java | 30 +++++++ pulsar-client-tools/pom.xml | 5 ++ .../pulsar/client/cli/PulsarClientTool.java | 13 ++- pulsar-testclient/pom.xml | 6 ++ .../testclient/LoadSimulationClient.java | 7 +- .../pulsar/testclient/PerfClientUtils.java | 2 + .../testclient/PerformanceBaseArguments.java | 5 ++ .../testclient/PerformanceConsumer.java | 2 + .../testclient/PerformanceProducer.java | 2 + .../pulsar/testclient/PerformanceReader.java | 2 + .../testclient/PerformanceTransaction.java | 2 + .../PerformanceBaseArgumentsTest.java | 80 +++++++++++++++++-- src/check-binary-license.sh | 2 +- 13 files changed, 149 insertions(+), 9 deletions(-) diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java index 8b32ad906eac4..f84f344ba5d69 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java @@ -73,6 +73,7 @@ public void testInitialization() throws InterruptedException, ExecutionException Properties properties = new Properties(); properties.setProperty("serviceUrl", brokerUrl.toString()); properties.setProperty("useTls", "false"); + properties.setProperty("memoryLimit", "10M"); String tenantName = UUID.randomUUID().toString(); @@ -94,6 +95,7 @@ public void testInitialization() throws InterruptedException, ExecutionException String[] args = { "consume", "-t", "Exclusive", "-s", "sub-name", "-n", Integer.toString(numberOfMessages), "--hex", "-r", "30", topicName }; Assert.assertEquals(pulsarClientToolConsumer.run(args), 0); + Assert.assertEquals(pulsarClientToolConsumer.rootParams.memoryLimit, 10 * 1024 * 1024); future.complete(null); } catch (Throwable t) { future.completeExceptionally(t); @@ -108,6 +110,7 @@ public void testInitialization() throws InterruptedException, ExecutionException String[] args = { "produce", "--messages", "Have a nice day", "-n", Integer.toString(numberOfMessages), "-r", "20", "-p", "key1=value1", "-p", "key2=value2", "-k", "partition_key", topicName }; Assert.assertEquals(pulsarClientToolProducer.run(args), 0); + Assert.assertEquals(pulsarClientToolProducer.rootParams.memoryLimit, 10 * 1024 * 1024); future.get(); } @@ -342,22 +345,49 @@ public void testArgs() throws Exception { final String message = "test msg"; final int numberOfMessages = 1; final String topicName = getTopicWithRandomSuffix("test-topic"); + final String memoryLimitArg = "10M"; String[] args = {"--url", url, "--auth-plugin", authPlugin, "--auth-params", authParams, "--tlsTrustCertsFilePath", CA_CERT_FILE_PATH, + "--memory-limit", memoryLimitArg, "produce", "-m", message, "-n", Integer.toString(numberOfMessages), topicName}; pulsarClientTool.jcommander.parse(args); assertEquals(pulsarClientTool.rootParams.getTlsTrustCertsFilePath(), CA_CERT_FILE_PATH); assertEquals(pulsarClientTool.rootParams.getAuthParams(), authParams); assertEquals(pulsarClientTool.rootParams.getAuthPluginClassName(), authPlugin); + assertEquals(pulsarClientTool.rootParams.getMemoryLimit(), 10 * 1024 * 1024); assertEquals(pulsarClientTool.rootParams.getServiceURL(), url); assertNull(pulsarClientTool.rootParams.getProxyServiceURL()); assertNull(pulsarClientTool.rootParams.getProxyProtocol()); } + @Test(timeOut = 20000) + public void testMemoryLimitArgShortName() throws Exception { + PulsarClientTool pulsarClientTool = new PulsarClientTool(new Properties()); + final String url = "pulsar+ssl://localhost:6651"; + final String authPlugin = "org.apache.pulsar.client.impl.auth.AuthenticationTls"; + final String authParams = String.format("tlsCertFile:%s,tlsKeyFile:%s", getTlsFileForClient("admin.cert"), + getTlsFileForClient("admin.key-pk8")); + final String message = "test msg"; + final int numberOfMessages = 1; + final String topicName = getTopicWithRandomSuffix("test-topic"); + final String memoryLimitArg = "10M"; + + String[] args = {"--url", url, + "--auth-plugin", authPlugin, + "--auth-params", authParams, + "--tlsTrustCertsFilePath", CA_CERT_FILE_PATH, + "-ml", memoryLimitArg, + "produce", "-m", message, + "-n", Integer.toString(numberOfMessages), topicName}; + + pulsarClientTool.jcommander.parse(args); + assertEquals(pulsarClientTool.rootParams.getMemoryLimit(), 10 * 1024 * 1024); + } + @Test public void testParsingProxyServiceUrlAndProxyProtocolFromProperties() throws Exception { Properties properties = new Properties(); diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 136d9a596ae2e..4a35523131ad2 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -67,6 +67,11 @@ pulsar-client-messagecrypto-bc ${project.version} + + ${project.groupId} + pulsar-cli-utils + ${project.version} + org.asynchttpclient async-http-client 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 c64d80f380b9f..4057bbe9fdfd8 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 @@ -32,6 +32,7 @@ import lombok.Getter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.cli.converters.ByteUnitToLongConverter; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; @@ -40,7 +41,6 @@ import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException; import org.apache.pulsar.client.api.SizeUnit; - public class PulsarClientTool { @Getter @@ -76,6 +76,10 @@ public static class RootParams { @Parameter(names = { "--tlsTrustCertsFilePath" }, description = "File path to client trust certificates") String tlsTrustCertsFilePath; + + @Parameter(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " + + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) + long memoryLimit = 0L; } protected RootParams rootParams; @@ -151,6 +155,11 @@ protected void initRootParamsFromProperties(Properties properties) { this.rootParams.authParams = properties.getProperty("authParams"); this.rootParams.tlsTrustCertsFilePath = properties.getProperty("tlsTrustCertsFilePath"); this.rootParams.proxyServiceURL = StringUtils.trimToNull(properties.getProperty("proxyServiceUrl")); + // setting memory limit + this.rootParams.memoryLimit = StringUtils.isNotEmpty(properties.getProperty("memoryLimit")) + ? new ByteUnitToLongConverter("memoryLimit").convert(properties.getProperty("memoryLimit")) + : this.rootParams.memoryLimit; + String proxyProtocolString = StringUtils.trimToNull(properties.getProperty("proxyProtocol")); if (proxyProtocolString != null) { try { @@ -165,7 +174,7 @@ protected void initRootParamsFromProperties(Properties properties) { private void updateConfig() throws UnsupportedAuthenticationException { ClientBuilder clientBuilder = PulsarClient.builder() - .memoryLimit(0, SizeUnit.BYTES); + .memoryLimit(rootParams.memoryLimit, SizeUnit.BYTES); Authentication authentication = null; if (isNotBlank(this.rootParams.authPluginClassName)) { authentication = AuthenticationFactory.create(rootParams.authPluginClassName, rootParams.authParams); diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 598fe477be1be..a13b2db33104b 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -85,6 +85,12 @@ ${project.version} + + ${project.groupId} + pulsar-cli-utils + ${project.version} + + commons-configuration commons-configuration 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 64330ae2eeea1..982c71ce6a5f4 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 @@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import org.apache.pulsar.cli.converters.ByteUnitToLongConverter; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -180,6 +181,10 @@ private static class MainArguments { @Parameter(names = { "--service-url" }, description = "Pulsar Service URL", required = true) public String serviceURL; + + @Parameter(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. @@ -318,7 +323,7 @@ public LoadSimulationClient(final MainArguments arguments) throws Exception { .serviceHttpUrl(arguments.serviceURL) .build(); client = PulsarClient.builder() - .memoryLimit(0, SizeUnit.BYTES) + .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) .serviceUrl(arguments.serviceURL) .connectionsPerBroker(4) .ioThreads(Runtime.getRuntime().availableProcessors()) 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 b312ceb6e3eff..3b44023ef503e 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 @@ -30,6 +30,7 @@ import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SizeUnit; import org.apache.pulsar.common.util.DirectMemoryUtils; import org.slf4j.Logger; @@ -66,6 +67,7 @@ public static ClientBuilder createClientBuilderFromArguments(PerformanceBaseArgu throws PulsarClientException.UnsupportedAuthenticationException { ClientBuilder clientBuilder = PulsarClient.builder() + .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) .serviceUrl(arguments.serviceURL) .connectionsPerBroker(arguments.maxConnections) .ioThreads(arguments.ioThreads) 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 5ae79fb0bf9a4..bc4ab003c4670 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 @@ -28,6 +28,7 @@ import java.util.Properties; import lombok.SneakyThrows; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.cli.converters.ByteUnitToLongConverter; import org.apache.pulsar.client.api.ProxyProtocol; /** @@ -103,6 +104,10 @@ public abstract class PerformanceBaseArguments { @Parameter(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) public String deprecatedAuthPluginClassName; + @Parameter(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 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 59dabc9302622..9bd74be3aa859 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 @@ -47,6 +47,7 @@ import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SizeUnit; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; @@ -229,6 +230,7 @@ public static void main(String[] args) throws Exception { long testEndTime = startTime + (long) (arguments.testTime * 1e9); ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) + .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) .enableTransaction(arguments.isEnableTransaction); PulsarClient pulsarClient = clientBuilder.build(); 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 63e3e2ec6fd23..e57d6ca225123 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 @@ -69,6 +69,7 @@ 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.SizeUnit; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; @@ -512,6 +513,7 @@ private static void runProducer(int producerId, ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) + .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) .enableTransaction(arguments.isEnableTransaction); client = clientBuilder.build(); 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 ed5cc37644a31..6174caad1f938 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 @@ -40,6 +40,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.ReaderListener; +import org.apache.pulsar.client.api.SizeUnit; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; @@ -140,6 +141,7 @@ public static void main(String[] args) throws Exception { }; ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) + .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) .enableTls(arguments.useTls); PulsarClient pulsarClient = clientBuilder.build(); 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 469e6ab1f3fd6..3b422452d6401 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 @@ -58,6 +58,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.SizeUnit; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; @@ -223,6 +224,7 @@ public static void main(String[] args) } ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) + .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) .enableTransaction(!arguments.isDisableTransaction); PulsarClient client = clientBuilder.build(); 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 42c93be343074..699f138bfdaa8 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 @@ -18,20 +18,20 @@ */ package org.apache.pulsar.testclient; +import static org.apache.pulsar.client.api.ProxyProtocol.SNI; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; - import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import static org.apache.pulsar.client.api.ProxyProtocol.SNI; -import static org.testng.Assert.fail; - - public class PerformanceBaseArgumentsTest { @Test @@ -158,4 +158,74 @@ public void fillArgumentsFromProperties(Properties prop) { tempConfigFile.delete(); } } + + @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} + }; + } + + @Test(dataProvider = "memoryLimitCliArgumentProvider") + public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimit) { + for (String cmd : List.of( + "pulsar-perf read", + "pulsar-perf produce", + "pulsar-perf consume", + "pulsar-perf transaction" + )) { + // Arrange + AtomicBoolean called = new AtomicBoolean(); + final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments() { + @Override + public void fillArgumentsFromProperties(Properties prop) { + called.set(true); + } + }; + baseArgument.confFile = "./src/test/resources/perf_client1.conf"; + + // Act + baseArgument.parseCLI(cmd, cliArgs); + + // Assert + assertEquals(baseArgument.memoryLimit, expectedMemoryLimit); + } + } + + @DataProvider(name = "invalidMemoryLimitCliArgumentProvider") + public Object[][] invalidMemoryLimitCliArgumentProvider() { + return new Object[][] { + { new String[]{"-ml","-1"}}, + { new String[]{"-ml","1C"}}, + { new String[]{"--memory-limit", "1Q"}} + }; + } + + @Test + public void testMemoryLimitCliArgumentDefault() { + for (String cmd : List.of( + "pulsar-perf read", + "pulsar-perf produce", + "pulsar-perf consume", + "pulsar-perf transaction" + )) { + // Arrange + AtomicBoolean called = new AtomicBoolean(); + final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments() { + @Override + public void fillArgumentsFromProperties(Properties prop) { + called.set(true); + } + }; + baseArgument.confFile = "./src/test/resources/perf_client1.conf"; + + // Act + baseArgument.parseCLI(cmd, new String[]{}); + + // Assert + assertEquals(baseArgument.memoryLimit, 0L); + } + } } diff --git a/src/check-binary-license.sh b/src/check-binary-license.sh index 3a6d266345f30..4b48da2061c3a 100755 --- a/src/check-binary-license.sh +++ b/src/check-binary-license.sh @@ -41,7 +41,7 @@ if [ -z $TARBALL ]; then exit 1 fi -JARS=$(tar -tf $TARBALL | grep '\.jar' | grep -v 'trino/' | grep -v '/examples/' | grep -v '/instances/' | grep -v pulsar-client | grep -v pulsar-common | grep -v pulsar-package | grep -v pulsar-websocket | grep -v bouncy-castle-bc | sed 's!.*/!!' | sort) +JARS=$(tar -tf $TARBALL | grep '\.jar' | grep -v 'trino/' | grep -v '/examples/' | grep -v '/instances/' | grep -v pulsar-client | grep -v pulsar-cli-utils | grep -v pulsar-common | grep -v pulsar-package | grep -v pulsar-websocket | grep -v bouncy-castle-bc | sed 's!.*/!!' | sort) LICENSEPATH=$(tar -tf $TARBALL | awk '/^[^\/]*\/LICENSE/') LICENSE=$(tar -O -xf $TARBALL "$LICENSEPATH") From 84204207c19cad0bf9986088af2bbd954b85288b Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 10 Oct 2023 23:45:10 +0800 Subject: [PATCH 03/37] [fix][sec] Fix MultiRoles token provider when using anonymous clients (#21338) Co-authored-by: Lari Hotari --- .../MultiRolesTokenAuthorizationProvider.java | 45 +++++++++++-------- ...tiRolesTokenAuthorizationProviderTest.java | 31 ++++++++++--- 2 files changed, 50 insertions(+), 26 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java index db5f4f18e8cc3..6376b60217fef 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java @@ -97,7 +97,7 @@ public CompletableFuture isSuperUser(String role, AuthenticationDataSou if (role != null && superUserRoles.contains(role)) { return CompletableFuture.completedFuture(true); } - Set roles = getRoles(authenticationData); + Set roles = getRoles(role, authenticationData); if (roles.isEmpty()) { return CompletableFuture.completedFuture(false); } @@ -112,7 +112,7 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, S if (isSuperUser) { return CompletableFuture.completedFuture(true); } - Set roles = getRoles(authData); + Set roles = getRoles(role, authData); if (roles.isEmpty()) { return CompletableFuture.completedFuture(false); } @@ -143,7 +143,11 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, S }); } - private Set getRoles(AuthenticationDataSource authData) { + private Set getRoles(String role, AuthenticationDataSource authData) { + if (authData == null) { + return Collections.singleton(role); + } + String token = null; if (authData.hasDataFromCommand()) { @@ -192,9 +196,9 @@ private Set getRoles(AuthenticationDataSource authData) { return Collections.emptySet(); } - public CompletableFuture authorize(AuthenticationDataSource authenticationData, Function> authorizeFunc) { - Set roles = getRoles(authenticationData); + public CompletableFuture authorize(String role, AuthenticationDataSource authenticationData, + Function> authorizeFunc) { + Set roles = getRoles(role, authenticationData); if (roles.isEmpty()) { return CompletableFuture.completedFuture(false); } @@ -212,7 +216,7 @@ public CompletableFuture authorize(AuthenticationDataSource authenticat @Override public CompletableFuture canProduceAsync(TopicName topicName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.canProduceAsync(topicName, r, authenticationData)); + return authorize(role, authenticationData, r -> super.canProduceAsync(topicName, r, authenticationData)); } /** @@ -227,7 +231,7 @@ public CompletableFuture canProduceAsync(TopicName topicName, String ro public CompletableFuture canConsumeAsync(TopicName topicName, String role, AuthenticationDataSource authenticationData, String subscription) { - return authorize(authenticationData, r -> super.canConsumeAsync(topicName, r, authenticationData, + return authorize(role, authenticationData, r -> super.canConsumeAsync(topicName, r, authenticationData, subscription)); } @@ -244,25 +248,27 @@ public CompletableFuture canConsumeAsync(TopicName topicName, String ro @Override public CompletableFuture canLookupAsync(TopicName topicName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.canLookupAsync(topicName, r, authenticationData)); + return authorize(role, authenticationData, r -> super.canLookupAsync(topicName, r, authenticationData)); } @Override public CompletableFuture allowFunctionOpsAsync(NamespaceName namespaceName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.allowFunctionOpsAsync(namespaceName, r, authenticationData)); + return authorize(role, authenticationData, + r -> super.allowFunctionOpsAsync(namespaceName, r, authenticationData)); } @Override public CompletableFuture allowSourceOpsAsync(NamespaceName namespaceName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.allowSourceOpsAsync(namespaceName, r, authenticationData)); + return authorize(role, authenticationData, + r -> super.allowSourceOpsAsync(namespaceName, r, authenticationData)); } @Override public CompletableFuture allowSinkOpsAsync(NamespaceName namespaceName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.allowSinkOpsAsync(namespaceName, r, authenticationData)); + return authorize(role, authenticationData, r -> super.allowSinkOpsAsync(namespaceName, r, authenticationData)); } @Override @@ -270,7 +276,7 @@ public CompletableFuture allowTenantOperationAsync(String tenantName, String role, TenantOperation operation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowTenantOperationAsync(tenantName, r, operation, authData)); + return authorize(role, authData, r -> super.allowTenantOperationAsync(tenantName, r, operation, authData)); } @Override @@ -278,7 +284,8 @@ public CompletableFuture allowNamespaceOperationAsync(NamespaceName nam String role, NamespaceOperation operation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowNamespaceOperationAsync(namespaceName, r, operation, authData)); + return authorize(role, authData, + r -> super.allowNamespaceOperationAsync(namespaceName, r, operation, authData)); } @Override @@ -287,8 +294,8 @@ public CompletableFuture allowNamespacePolicyOperationAsync(NamespaceNa PolicyOperation operation, String role, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowNamespacePolicyOperationAsync(namespaceName, policy, operation, r, - authData)); + return authorize(role, authData, + r -> super.allowNamespacePolicyOperationAsync(namespaceName, policy, operation, r, authData)); } @Override @@ -296,7 +303,7 @@ public CompletableFuture allowTopicOperationAsync(TopicName topicName, String role, TopicOperation operation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowTopicOperationAsync(topicName, r, operation, authData)); + return authorize(role, authData, r -> super.allowTopicOperationAsync(topicName, r, operation, authData)); } @Override @@ -305,7 +312,7 @@ public CompletableFuture allowTopicPolicyOperationAsync(TopicName topic PolicyName policyName, PolicyOperation policyOperation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowTopicPolicyOperationAsync(topicName, r, policyName, policyOperation, - authData)); + return authorize(role, authData, + r -> super.allowTopicPolicyOperationAsync(topicName, r, policyName, policyOperation, authData)); } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java index f0a857bdd695d..4b67f52075ca9 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java @@ -24,6 +24,8 @@ import io.jsonwebtoken.Jwts; import io.jsonwebtoken.SignatureAlgorithm; import java.util.Properties; +import java.util.function.Function; +import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; @@ -61,18 +63,18 @@ public String getHttpHeader(String name) { } }; - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { if (role.equals(userB)) { return CompletableFuture.completedFuture(true); // only userB has permission } return CompletableFuture.completedFuture(false); }).get()); - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { return CompletableFuture.completedFuture(true); // all users has permission }).get()); - assertFalse(provider.authorize(ads, role -> { + assertFalse(provider.authorize("test", ads, role -> { return CompletableFuture.completedFuture(false); // all users has no permission }).get()); } @@ -100,7 +102,7 @@ public String getHttpHeader(String name) { } }; - assertFalse(provider.authorize(ads, role -> CompletableFuture.completedFuture(false)).get()); + assertFalse(provider.authorize("test", ads, role -> CompletableFuture.completedFuture(false)).get()); } @Test @@ -127,7 +129,7 @@ public String getHttpHeader(String name) { } }; - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { if (role.equals(testRole)) { return CompletableFuture.completedFuture(true); } @@ -135,6 +137,21 @@ public String getHttpHeader(String name) { }).get()); } + @Test + public void testMultiRolesAuthzWithAnonymousUser() throws Exception { + @Cleanup + MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + + Function> authorizeFunc = (String role) -> { + if (role.equals("test-role")) { + return CompletableFuture.completedFuture(true); + } + return CompletableFuture.completedFuture(false); + }; + assertTrue(provider.authorize("test-role", null, authorizeFunc).get()); + assertFalse(provider.authorize("test-role-x", null, authorizeFunc).get()); + } + @Test public void testMultiRolesNotFailNonJWT() throws Exception { String token = "a-non-jwt-token"; @@ -157,7 +174,7 @@ public String getHttpHeader(String name) { } }; - assertFalse(provider.authorize(ads, role -> CompletableFuture.completedFuture(false)).get()); + assertFalse(provider.authorize("test", ads, role -> CompletableFuture.completedFuture(false)).get()); } @Test @@ -192,7 +209,7 @@ public String getHttpHeader(String name) { } }; - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { if (role.equals(testRole)) { return CompletableFuture.completedFuture(true); } From 61a7adf08b14067500f9bd17b6da824ba58e9707 Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Wed, 11 Oct 2023 11:07:50 +0800 Subject: [PATCH 04/37] [fix] [bk-client] Fix bk client MinNumRacksPerWriteQuorum and EnforceMinNumRacksPerWriteQuorum not work problem. (#21327) --- .../broker/BookKeeperClientFactoryImpl.java | 5 ++++- .../BookKeeperClientFactoryImplTest.java | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) 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 0ecca75595603..e5293cee24e4a 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 @@ -219,7 +219,7 @@ static void setDefaultEnsemblePlacementPolicy( } } - private void setEnsemblePlacementPolicy(ClientConfiguration bkConf, ServiceConfiguration conf, MetadataStore store, + static void setEnsemblePlacementPolicy(ClientConfiguration bkConf, ServiceConfiguration conf, MetadataStore store, Class policyClass) { bkConf.setEnsemblePlacementPolicy(policyClass); bkConf.setProperty(BookieRackAffinityMapping.METADATA_STORE_INSTANCE, store); @@ -227,6 +227,9 @@ private void setEnsemblePlacementPolicy(ClientConfiguration bkConf, ServiceConfi bkConf.setProperty(REPP_DNS_RESOLVER_CLASS, conf.getProperties().getProperty(REPP_DNS_RESOLVER_CLASS, BookieRackAffinityMapping.class.getName())); + bkConf.setMinNumRacksPerWriteQuorum(conf.getBookkeeperClientMinNumRacksPerWriteQuorum()); + bkConf.setEnforceMinNumRacksPerWriteQuorum(conf.isBookkeeperClientEnforceMinNumRacksPerWriteQuorum()); + bkConf.setProperty(NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, conf.getProperties().getProperty( NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java index 0dea84e727a88..3c0e4d0c409df 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.bookie.rackawareness.BookieRackAffinityMapping; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy; import org.testng.annotations.Test; /** @@ -152,6 +153,24 @@ public void testSetDefaultEnsemblePlacementPolicyRackAwareEnabledChangedValues() assertEquals(20, bkConf.getMinNumRacksPerWriteQuorum()); } + @Test + public void testSetEnsemblePlacementPolicys() { + ClientConfiguration bkConf = new ClientConfiguration(); + ServiceConfiguration conf = new ServiceConfiguration(); + conf.setBookkeeperClientMinNumRacksPerWriteQuorum(3); + conf.setBookkeeperClientEnforceMinNumRacksPerWriteQuorum(true); + + MetadataStore store = mock(MetadataStore.class); + + BookKeeperClientFactoryImpl.setEnsemblePlacementPolicy( + bkConf, + conf, + store, + ZkIsolatedBookieEnsemblePlacementPolicy.class); + assertEquals(bkConf.getMinNumRacksPerWriteQuorum(), 3); + assertTrue(bkConf.getEnforceMinNumRacksPerWriteQuorum()); + } + @Test public void testSetDiskWeightBasedPlacementEnabled() { BookKeeperClientFactoryImpl factory = new BookKeeperClientFactoryImpl(); From eb9fa63d6bcaa4e1cbc4b87e36ead5a3ff6c44ae Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 11 Oct 2023 10:13:26 +0300 Subject: [PATCH 05/37] [fix][ml] Make mlOwnershipChecker asynchronous so that it doesn't block/deadlock threads (#21333) --- .../mledger/ManagedLedgerFactory.java | 2 +- .../mledger/impl/ManagedCursorImpl.java | 57 ++++++++++++------- .../impl/ManagedLedgerFactoryImpl.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 4 +- .../mledger/impl/ShadowManagedLedgerImpl.java | 3 +- .../mledger/impl/ManagedLedgerTest.java | 2 +- .../pulsar/broker/service/BrokerService.java | 19 ++++--- .../impl/MLPendingAckStoreProvider.java | 2 +- .../broker/admin/TopicPoliciesTest.java | 2 +- .../OwnerShipCacheForCurrentServerTest.java | 2 +- 10 files changed, 57 insertions(+), 38 deletions(-) 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 b1427bab80b22..e09fd84ea55f2 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 @@ -90,7 +90,7 @@ ManagedLedger open(String name, ManagedLedgerConfig config) * opaque context */ void asyncOpen(String name, ManagedLedgerConfig config, OpenLedgerCallback callback, - Supplier mlOwnershipChecker, Object ctx); + Supplier> mlOwnershipChecker, Object ctx); /** * Open a {@link ReadOnlyCursor} positioned to the earliest entry for the specified managed ledger. 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 7eccd70223b27..39f56a1ad604b 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 @@ -2685,32 +2685,47 @@ public void operationComplete(Void result, Stat stat) { } @Override - public void operationFailed(MetaStoreException e) { - if (e instanceof MetaStoreException.BadVersionException) { + public void operationFailed(MetaStoreException topLevelException) { + if (topLevelException instanceof MetaStoreException.BadVersionException) { log.warn("[{}] Failed to update cursor metadata for {} due to version conflict {}", - ledger.name, name, e.getMessage()); + ledger.name, name, topLevelException.getMessage()); // it means previous owner of the ml might have updated the version incorrectly. So, check // the ownership and refresh the version again. - if (ledger.mlOwnershipChecker != null && ledger.mlOwnershipChecker.get()) { - ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, - new MetaStoreCallback() { - @Override - public void operationComplete(ManagedCursorInfo info, Stat stat) { - updateCursorLedgerStat(info, stat); - } - - @Override - public void operationFailed(MetaStoreException e) { - if (log.isDebugEnabled()) { - log.debug( - "[{}] Failed to refresh cursor metadata-version for {} due " - + "to {}", ledger.name, name, e.getMessage()); - } - } - }); + if (ledger.mlOwnershipChecker != null) { + ledger.mlOwnershipChecker.get().whenComplete((hasOwnership, t) -> { + if (t == null && hasOwnership) { + ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, + new MetaStoreCallback<>() { + @Override + public void operationComplete(ManagedCursorInfo info, Stat stat) { + updateCursorLedgerStat(info, stat); + // fail the top level call so that the caller can retry + callback.operationFailed(topLevelException); + } + + @Override + public void operationFailed(MetaStoreException e) { + if (log.isDebugEnabled()) { + log.debug( + "[{}] Failed to refresh cursor metadata-version " + + "for {} due to {}", ledger.name, name, + e.getMessage()); + } + // fail the top level call so that the caller can retry + callback.operationFailed(topLevelException); + } + }); + } else { + // fail the top level call so that the caller can retry + callback.operationFailed(topLevelException); + } + }); + } else { + callback.operationFailed(topLevelException); } + } else { + callback.operationFailed(topLevelException); } - callback.operationFailed(e); } }); } 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 9107b76c88a28..03605bf6e8519 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 @@ -330,7 +330,7 @@ public void asyncOpen(String name, OpenLedgerCallback callback, Object ctx) { @Override public void asyncOpen(final String name, final ManagedLedgerConfig config, final OpenLedgerCallback callback, - Supplier mlOwnershipChecker, final Object ctx) { + Supplier> mlOwnershipChecker, final Object ctx) { if (closed) { callback.openLedgerFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); return; 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 cc0020dcd9eb1..e349bf508086f 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 @@ -232,7 +232,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private static final Random random = new Random(System.currentTimeMillis()); private long maximumRolloverTimeMs; - protected final Supplier mlOwnershipChecker; + protected final Supplier> mlOwnershipChecker; volatile PositionImpl lastConfirmedEntry; @@ -336,7 +336,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper } public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - final String name, final Supplier mlOwnershipChecker) { + final String name, final Supplier> mlOwnershipChecker) { this.factory = factory; this.bookKeeper = bookKeeper; this.config = config; 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 b33dd87543f77..8b2742d958783 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 @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.AsyncCallback; @@ -50,7 +51,7 @@ public class ShadowManagedLedgerImpl extends ManagedLedgerImpl { public ShadowManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - String name, final Supplier mlOwnershipChecker) { + String name, final Supplier> mlOwnershipChecker) { super(factory, bookKeeper, store, config, scheduledExecutor, name, mlOwnershipChecker); this.sourceMLName = config.getShadowSourceName(); } 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 5fc2da22b661e..cd61e00ccaa8e 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 @@ -3389,7 +3389,7 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { @Override public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } - }, checkOwnershipFlag ? () -> true : null, null); + }, checkOwnershipFlag ? () -> CompletableFuture.completedFuture(true) : null, null); latch.await(); } 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 388d9de84f833..d03a94a0563ab 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 @@ -1748,7 +1748,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { topicFuture.completeExceptionally(new PersistenceException(exception)); } } - }, () -> isTopicNsOwnedByBroker(topicName), null); + }, () -> isTopicNsOwnedByBrokerAsync(topicName), null); }).exceptionally((exception) -> { log.warn("[{}] Failed to get topic configuration: {}", topic, exception.getMessage(), exception); @@ -2136,13 +2136,16 @@ public void monitorBacklogQuota() { }); } - public boolean isTopicNsOwnedByBroker(TopicName topicName) { - try { - return pulsar.getNamespaceService().isServiceUnitOwned(topicName); - } catch (Exception e) { - log.warn("Failed to check the ownership of the topic: {}, {}", topicName, e.getMessage()); - } - return false; + public CompletableFuture isTopicNsOwnedByBrokerAsync(TopicName topicName) { + return pulsar.getNamespaceService().isServiceUnitOwnedAsync(topicName) + .handle((hasOwnership, t) -> { + if (t == null) { + return hasOwnership; + } else { + log.warn("Failed to check the ownership of the topic: {}, {}", topicName, t.getMessage()); + return false; + } + }); } public CompletableFuture checkTopicNsOwnership(final String topic) { 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 ecc6599ce52b5..5308648b80c1d 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 @@ -159,7 +159,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { , originPersistentTopic.getName(), subscription.getName(), exception); pendingAckStoreFuture.completeExceptionally(exception); } - }, () -> true, null); + }, () -> CompletableFuture.completedFuture(true), null); }).exceptionally(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); log.error("[{}] [{}] Failed to get managedLedger config when init pending ack store!", 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 faf141a5d1cf4..4e510a50f1098 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 @@ -176,7 +176,7 @@ public void testTopicPolicyInitialValueWithNamespaceAlreadyLoaded() throws Excep //load the nameserver, but topic is not init. log.info("lookup:{}",admin.lookups().lookupTopic(topic)); - assertTrue(pulsar.getBrokerService().isTopicNsOwnedByBroker(topicName)); + assertTrue(pulsar.getBrokerService().isTopicNsOwnedByBrokerAsync(topicName).join()); assertFalse(pulsar.getBrokerService().getTopics().containsKey(topic)); //make sure namespace policy reader is fully started. Awaitility.await().untilAsserted(()-> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java index e53d5c25bd2b5..22fa2c32b5655 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java @@ -76,7 +76,7 @@ public void testCreateTopicWithNotTopicNsOwnedBroker() { int verifiedBrokerNum = 0; for (PulsarService pulsarService : this.getPulsarServiceList()) { BrokerService bs = pulsarService.getBrokerService(); - if (bs.isTopicNsOwnedByBroker(TopicName.get(topicName))) { + if (bs.isTopicNsOwnedByBrokerAsync(TopicName.get(topicName)).join()) { continue; } verifiedBrokerNum ++; From aecdb03e0e64605d60f03d9b76f99c1136677dff Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:22:44 -0700 Subject: [PATCH 06/37] [improve][broker] use ConcurrentHashMap in ServiceUnitStateChannel and avoid recursive update error (#21282) --- .../channel/ServiceUnitStateChannelImpl.java | 90 +++++++++++-------- .../channel/ServiceUnitStateChannelTest.java | 10 +-- 2 files changed, 59 insertions(+), 41 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 d71513652e9b8..f7e09a2bec546 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 @@ -54,6 +54,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; @@ -67,6 +68,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.PulsarClusterMetadataSetup; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -97,7 +99,6 @@ 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.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.extended.SessionEvent; @@ -125,9 +126,9 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final PulsarService pulsar; private final ServiceConfiguration config; private final Schema schema; - private final ConcurrentOpenHashMap> getOwnerRequests; + private final Map> getOwnerRequests; private final String lookupServiceAddress; - private final ConcurrentOpenHashMap> cleanupJobs; + private final Map> cleanupJobs; private final StateChangeListeners stateChangeListeners; private ExtensibleLoadManagerImpl loadManager; private BrokerRegistry brokerRegistry; @@ -204,9 +205,8 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.config = pulsar.getConfig(); this.lookupServiceAddress = pulsar.getLookupServiceAddress(); this.schema = Schema.JSON(ServiceUnitStateData.class); - this.getOwnerRequests = ConcurrentOpenHashMap.>newBuilder().build(); - this.cleanupJobs = ConcurrentOpenHashMap.>newBuilder().build(); + this.getOwnerRequests = new ConcurrentHashMap<>(); + this.cleanupJobs = new ConcurrentHashMap<>(); this.stateChangeListeners = new StateChangeListeners(); this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateTombstoneDelayTimeInSeconds() * 1000; @@ -826,20 +826,28 @@ private boolean isTargetBroker(String broker) { } private CompletableFuture deferGetOwnerRequest(String serviceUnit) { - return getOwnerRequests - .computeIfAbsent(serviceUnit, k -> { - CompletableFuture future = new CompletableFuture<>(); - future.orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) - .whenComplete((v, e) -> { - if (e != null) { - getOwnerRequests.remove(serviceUnit, future); - log.warn("Failed to getOwner for serviceUnit:{}", - serviceUnit, e); - } + var requested = new MutableObject>(); + try { + return getOwnerRequests + .computeIfAbsent(serviceUnit, k -> { + CompletableFuture future = new CompletableFuture<>(); + requested.setValue(future); + return future; + }); + } finally { + var future = requested.getValue(); + if (future != null) { + future.orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) + .whenComplete((v, e) -> { + if (e != null) { + getOwnerRequests.remove(serviceUnit, future); + log.warn("Failed to getOwner for serviceUnit:{}", + serviceUnit, e); } - ); - return future; - }); + } + ); + } + } } private CompletableFuture closeServiceUnit(String serviceUnit) { @@ -1114,24 +1122,34 @@ private void handleBrokerDeletionEvent(String broker) { } private void scheduleCleanup(String broker, long delayInSecs) { - cleanupJobs.computeIfAbsent(broker, k -> { - Executor delayed = CompletableFuture - .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); - totalInactiveBrokerCleanupScheduledCnt++; - return CompletableFuture - .runAsync(() -> { - try { - doCleanup(broker); - } catch (Throwable e) { - log.error("Failed to run the cleanup job for the broker {}, " - + "totalCleanupErrorCnt:{}.", - broker, totalCleanupErrorCnt.incrementAndGet(), e); - } finally { - cleanupJobs.remove(broker); + var scheduled = new MutableObject>(); + try { + cleanupJobs.computeIfAbsent(broker, k -> { + Executor delayed = CompletableFuture + .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); + totalInactiveBrokerCleanupScheduledCnt++; + var future = CompletableFuture + .runAsync(() -> { + try { + doCleanup(broker); + } catch (Throwable e) { + log.error("Failed to run the cleanup job for the broker {}, " + + "totalCleanupErrorCnt:{}.", + broker, totalCleanupErrorCnt.incrementAndGet(), e); + } } - } - , delayed); - }); + , delayed); + scheduled.setValue(future); + return future; + }); + } finally { + var future = scheduled.getValue(); + if (future != null) { + future.whenComplete((v, ex) -> { + cleanupJobs.remove(broker); + }); + } + } log.info("Scheduled ownership cleanup for broker:{} with delay:{} secs. Pending clean jobs:{}.", broker, delayInSecs, cleanupJobs.size()); 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 a226df53e12f3..f9893ea3f63dc 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 @@ -60,6 +60,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -88,7 +89,6 @@ import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.policies.data.TopicType; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; @@ -1558,9 +1558,9 @@ public void testOverrideOrphanStateData() } - private static ConcurrentOpenHashMap>> getOwnerRequests( + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { - return (ConcurrentOpenHashMap>>) + return (ConcurrentHashMap>>) FieldUtils.readDeclaredField(channel, "getOwnerRequests", true); } @@ -1577,9 +1577,9 @@ private static long getLastMetadataSessionEventTimestamp(ServiceUnitStateChannel FieldUtils.readField(channel, "lastMetadataSessionEventTimestamp", true); } - private static ConcurrentOpenHashMap> getCleanupJobs( + private static ConcurrentHashMap> getCleanupJobs( ServiceUnitStateChannel channel) throws IllegalAccessException { - return (ConcurrentOpenHashMap>) + return (ConcurrentHashMap>) FieldUtils.readField(channel, "cleanupJobs", true); } From 1a352f178590a210a9200dfee135089155c2f168 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 12 Oct 2023 09:42:02 +0800 Subject: [PATCH 07/37] [improve] [pip] PIP-299: Stop dispatch messages if the individual acks will be lost in the persistent storage (#21118) - DISCUSS: https://lists.apache.org/thread/2fzo1gnlyd1t5o80g1polbvbx4699r09 - VOTE: https://lists.apache.org/thread/j3k97599b3qq0lhmgyp5f93o6ny99cdt --- pip/pip-299.md | 81 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 pip/pip-299.md diff --git a/pip/pip-299.md b/pip/pip-299.md new file mode 100644 index 0000000000000..342d663cee984 --- /dev/null +++ b/pip/pip-299.md @@ -0,0 +1,81 @@ +# Background knowledge + +The config `managedLedgerMaxUnackedRangesToPersist` + +Indicates the 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 maximum number of ranges is reached, the information will only be tracked in memory, and messages will be redelivered in case of crashes. + +The cursor metadata contains the following three data: +- Subscription properties(Usually, this is small); this data part only persists to the ZK node. +- The last sequence ID of each producer. It only exists for the cursor `pulsar.dedup`. If a topic has many, many producers, this part of the data will be large. See [PIP-6:-Guaranteed-Message-Deduplication](https://github.com/apache/pulsar/wiki/PIP-6:-Guaranteed-Message-Deduplication) for more details. +- Individual Deleted Messages(including the acknowledgment of batched messages). This part of the data occupies most of the cursor metadata's space, which is the focus of this proposal. + +Differ with Kafka: Pulsar supports [individual acknowledgment](https://pulsar.apache.org/docs/2.11.x/concepts-messaging/#acknowledgment) (just like ack `{pos-1, pos-3, pos-5}`), so instead of a pointer(acknowledged on the left and un-acknowledged on the right), Pulsar needs to persist the acknowledgment state of each message, we call these records `Individual Deleted Messages.` + +The current persistence mechanism of the cursor metadata(including `Individual Deleted Messages`) works like this: +1. Write the data of cursor metadata(including `Individual Deleted Messages`) to BK in one Entry; by default, the maximum size of the Entry is 5MB. +2. Write the data of cursor metadata(optional to include `Individual Deleted Messages`) to the Metadata Store(such as ZK) if BK-Write fails; data of a Metadata Store Node that is less than 10MB is recommended. Since writing large chunks of data to the Metadata Store frequently makes the Metadata Store work unstable, this is only a backstop measure. + +Is 5MB enough? `Individual Deleted Messages` consists of Position_Rang(each Position_Rang occupies 32 bytes; the implementation will not be explained in this proposal). This means that the Broker can persist `5m / 32bytes` number of Position_Rang for each Subscription, and there is an additional compression mechanism at work, so it is sufficient for almost all scenarios except the following three scenarios: +- Client Miss Acknowledges: Clients receive many messages, and ack some of them, the rest still need to be acknowledged due to errors or other reasons. As time goes on, more and more records will be staying there. +- Delay Messages: Long-delayed and short-delayed messages are mixed, with only the short-delayed message successfully consumed and the long-delayed message not delivered. As time goes on, more and more records will be staying there. +- Large Number of Consumers: If the number of consumers is large and each has some discrete ack records, all add up to a large number. +- Large Number of Producers: If the number of producers is large, there might be a large data of Last Sequence ID to persist. This scenario only exists on the `pulsar.dedup` cursor. + +The config `managedLedgerMaxUnackedRangesToPersist` +If the cursor metadata is too large to persist, the Broker will persist only part of the data according to the following priorities. +- Subscription Properties. This part can't be split up; persist will fail if this part is too large to persist. +- Last sequence ID of producers. This part can't be split up; persist will fail if this part is too large to persist. +- Individual Deleted Message. If it is too large, only one part persists, and then the other part is maintained only in memory + +# Motivation + +Since the frequent persistence of `Individual Deleted Messages` will magnify the amount of BK Written and increase the latency of ack-response, the Broker does not immediately persist it when receiving a consumer's acknowledgment but persists it regularly. + +The data of cursor metadata is recommended to be less than 5MB; if a subscription's `Individual Deleted Messages` data is too large to persist, as the program grows for a long time, there will be more and more non-persistent data. Eventually, there will be an unacceptable amount of repeated consumption of messages when the Broker restarts. + +# Goal + +## In Scope + +To avoid repeated consumption due to the cursor metadata being too large to persist. + +## Out of Scope + +This proposal will not care about this scenario: if so many producers make the metadata of cursor `pulsar.dedup` cannot persist, the task `Take Deduplication Snapshot` will be in vain due to the inability to persist. + +# High-Level Design + +Provide a new config named `dispatcherPauseOnAckStatePersistentEnabled`(default value is `false`) for a new feature: stop dispatch messages to clients when reaching the limitation `managedLedgerMaxUnackedRangesToPersist`. +- If the user does not care about that Individual Deleted Messages can not be fully persistent, resulting in a large number of repeated message consumption, then it can be set to `false`. +- If the user cares about repeated consumption, at can accept a decline in consumption speed when cursor metadata is too large to persist, it can be set to `true`. + + +# Detailed Design +### Public API + +**broker.conf** +``` +/** + * After enabling this feature, Pulsar will stop delivery messages to clients if the cursor metadata is too large to persist, it will help to reduce the duplicates caused by the ack state that can not be fully persistent. Default "false". + */ +boolean dispatcherPauseOnAckStatePersistentEnabled; +``` + +**SubscriptionStats** +```java +/** + * After enabling the feature "dispatcherPauseOnAckStatePersistentEnabled", return "true" if the cursor metadata is too large to persist, else return "false". + * Always return "false" if disabled the feature "dispatcherPauseOnAckStatePersistentEnabled". + */ +boolean isBlockedOnAckStatePersistent(); +``` + +## Design & Implementation Details + +Cache the range count of the Individual Deleted Messages in the memory when doing persist cursor metadata to BK. Stuck delivery messages to clients if reaching the limitation `managedLedgerMaxUnackedRangesToPersist`. + +Since the cache will not be updated in time, the actual count will decrease when clients acknowledge messages(but it does not persist immediately, so the cached value does not update immediately), the cached value is an estimated value. + +# Metrics & Alert + +Nothing. From 421c98a1a5ec08941e794698dfc43a1a08d6e782 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 14 Oct 2023 10:45:10 +0300 Subject: [PATCH 08/37] [fix][ci] Fix docker image building by releasing more disk space before building (#21365) --- .github/actions/clean-disk/action.yml | 2 +- .github/workflows/pulsar-ci.yaml | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/.github/actions/clean-disk/action.yml b/.github/actions/clean-disk/action.yml index 8bcc5f1396802..d74c3f25fc64c 100644 --- a/.github/actions/clean-disk/action.yml +++ b/.github/actions/clean-disk/action.yml @@ -31,7 +31,7 @@ runs: directories=(/usr/local/lib/android /opt/ghc) if [[ "${{ inputs.mode }}" == "full" ]]; then # remove these directories only when mode is 'full' - directories+=(/usr/share/dotnet) + directories+=(/usr/share/dotnet /opt/hostedtoolcache/CodeQL) fi emptydir=/tmp/empty$$/ mkdir $emptydir diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 64b85cb14c580..e067e42f43c8d 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -746,6 +746,8 @@ jobs: - name: Clean Disk uses: ./.github/actions/clean-disk + with: + mode: full - name: Cache local Maven repository uses: actions/cache@v3 @@ -861,6 +863,7 @@ jobs: - name: Pulsar IO group: PULSAR_IO + clean_disk: true - name: Sql group: SQL @@ -872,6 +875,10 @@ jobs: - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm + - name: Clean Disk when needed + if: ${{ matrix.clean_disk }} + uses: ./.github/actions/clean-disk + - name: Setup ssh access to build runner VM # ssh access is enabled for builds in own forks if: ${{ github.repository != 'apache/pulsar' && github.event_name == 'pull_request' }} @@ -1072,6 +1079,7 @@ jobs: - name: Pulsar IO - Oracle group: PULSAR_IO_ORA + clean_disk: true steps: - name: checkout @@ -1080,6 +1088,10 @@ jobs: - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm + - name: Clean Disk when needed + if: ${{ matrix.clean_disk }} + uses: ./.github/actions/clean-disk + - name: Setup ssh access to build runner VM # ssh access is enabled for builds in own forks if: ${{ github.repository != 'apache/pulsar' && github.event_name == 'pull_request' }} From d6a56ad3094e9da89fa5f7cf00e0b7bbb36e6563 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 14 Oct 2023 13:31:27 +0300 Subject: [PATCH 09/37] [improve][broker] Optimize and clean up aggregation of topic stats (#21361) --- .../stats/NonPersistentTopicStatsImpl.java | 57 ++++++------------- .../policies/data/stats/TopicStatsImpl.java | 49 ++++++---------- 2 files changed, 35 insertions(+), 71 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java index fd643f0db7bf4..7710c27779b9a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentTopicStatsImpl.java @@ -155,8 +155,9 @@ public NonPersistentTopicStatsImpl add(NonPersistentTopicStats ts) { Objects.requireNonNull(stats); super.add(stats); this.msgDropRate += stats.msgDropRate; - for (int index = 0; index < stats.getNonPersistentPublishers().size(); index++) { - NonPersistentPublisherStats s = stats.getNonPersistentPublishers().get(index); + List publisherStats = stats.getNonPersistentPublishers(); + for (int index = 0; index < publisherStats.size(); index++) { + NonPersistentPublisherStats s = publisherStats.get(index); if (s.isSupportsPartialProducer() && s.getProducerName() != null) { ((NonPersistentPublisherStatsImpl) this.nonPersistentPublishersMap .computeIfAbsent(s.getProducerName(), key -> { @@ -181,46 +182,24 @@ public NonPersistentTopicStatsImpl add(NonPersistentTopicStats ts) { } } - if (this.getNonPersistentSubscriptions().size() != stats.getNonPersistentSubscriptions().size()) { - for (String subscription : stats.getNonPersistentSubscriptions().keySet()) { - NonPersistentSubscriptionStatsImpl subscriptionStats = new NonPersistentSubscriptionStatsImpl(); - this.getNonPersistentSubscriptions().put(subscription, subscriptionStats - .add((NonPersistentSubscriptionStatsImpl) - stats.getNonPersistentSubscriptions().get(subscription))); - } - } else { - for (String subscription : stats.getNonPersistentSubscriptions().keySet()) { - if (this.getNonPersistentSubscriptions().get(subscription) != null) { - ((NonPersistentSubscriptionStatsImpl) this.getNonPersistentSubscriptions().get(subscription)) - .add((NonPersistentSubscriptionStatsImpl) - stats.getNonPersistentSubscriptions().get(subscription)); - } else { - NonPersistentSubscriptionStatsImpl subscriptionStats = new NonPersistentSubscriptionStatsImpl(); - this.getNonPersistentSubscriptions().put(subscription, subscriptionStats - .add((NonPersistentSubscriptionStatsImpl) - stats.getNonPersistentSubscriptions().get(subscription))); - } - } + for (Map.Entry entry : stats.getNonPersistentSubscriptions() + .entrySet()) { + NonPersistentSubscriptionStatsImpl subscriptionStats = + (NonPersistentSubscriptionStatsImpl) this.getNonPersistentSubscriptions() + .computeIfAbsent(entry.getKey(), k -> new NonPersistentSubscriptionStatsImpl()); + subscriptionStats.add( + (NonPersistentSubscriptionStatsImpl) entry.getValue()); } - if (this.getNonPersistentReplicators().size() != stats.getNonPersistentReplicators().size()) { - for (String repl : stats.getNonPersistentReplicators().keySet()) { - NonPersistentReplicatorStatsImpl replStats = new NonPersistentReplicatorStatsImpl(); - this.getNonPersistentReplicators().put(repl, replStats - .add((NonPersistentReplicatorStatsImpl) stats.getNonPersistentReplicators().get(repl))); - } - } else { - for (String repl : stats.getNonPersistentReplicators().keySet()) { - if (this.getNonPersistentReplicators().get(repl) != null) { - ((NonPersistentReplicatorStatsImpl) this.getNonPersistentReplicators().get(repl)) - .add((NonPersistentReplicatorStatsImpl) stats.getNonPersistentReplicators().get(repl)); - } else { - NonPersistentReplicatorStatsImpl replStats = new NonPersistentReplicatorStatsImpl(); - this.getNonPersistentReplicators().put(repl, replStats - .add((NonPersistentReplicatorStatsImpl) stats.getNonPersistentReplicators().get(repl))); - } - } + for (Map.Entry entry : stats.getNonPersistentReplicators().entrySet()) { + NonPersistentReplicatorStatsImpl replStats = (NonPersistentReplicatorStatsImpl) + this.getNonPersistentReplicators().computeIfAbsent(entry.getKey(), k -> { + NonPersistentReplicatorStatsImpl r = new NonPersistentReplicatorStatsImpl(); + return r; + }); + replStats.add((NonPersistentReplicatorStatsImpl) entry.getValue()); } + return this; } 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 e50620fb22398..e022c885d663b 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 @@ -259,8 +259,9 @@ public TopicStatsImpl add(TopicStats ts) { topicMetricBean.value += v.value; }); - for (int index = 0; index < stats.getPublishers().size(); index++) { - PublisherStats s = stats.getPublishers().get(index); + List publisherStats = stats.getPublishers(); + for (int index = 0; index < publisherStats.size(); index++) { + PublisherStats s = publisherStats.get(index); if (s.isSupportsPartialProducer() && s.getProducerName() != null) { this.publishersMap.computeIfAbsent(s.getProducerName(), key -> { final PublisherStatsImpl newStats = new PublisherStatsImpl(); @@ -284,38 +285,22 @@ public TopicStatsImpl add(TopicStats ts) { } } - if (this.subscriptions.size() != stats.subscriptions.size()) { - for (String subscription : stats.subscriptions.keySet()) { - SubscriptionStatsImpl subscriptionStats = new SubscriptionStatsImpl(); - this.subscriptions.put(subscription, subscriptionStats.add(stats.subscriptions.get(subscription))); - } - } else { - for (String subscription : stats.subscriptions.keySet()) { - if (this.subscriptions.get(subscription) != null) { - this.subscriptions.get(subscription).add(stats.subscriptions.get(subscription)); - } else { - SubscriptionStatsImpl subscriptionStats = new SubscriptionStatsImpl(); - this.subscriptions.put(subscription, subscriptionStats.add(stats.subscriptions.get(subscription))); - } - } + for (Map.Entry entry : stats.subscriptions.entrySet()) { + SubscriptionStatsImpl subscriptionStats = + this.subscriptions.computeIfAbsent(entry.getKey(), k -> new SubscriptionStatsImpl()); + subscriptionStats.add(entry.getValue()); } - if (this.replication.size() != stats.replication.size()) { - for (String repl : stats.replication.keySet()) { - ReplicatorStatsImpl replStats = new ReplicatorStatsImpl(); - replStats.setConnected(true); - this.replication.put(repl, replStats.add(stats.replication.get(repl))); - } - } else { - for (String repl : stats.replication.keySet()) { - if (this.replication.get(repl) != null) { - this.replication.get(repl).add(stats.replication.get(repl)); - } else { - ReplicatorStatsImpl replStats = new ReplicatorStatsImpl(); - replStats.setConnected(true); - this.replication.put(repl, replStats.add(stats.replication.get(repl))); - } - } + + for (Map.Entry entry : stats.replication.entrySet()) { + ReplicatorStatsImpl replStats = + this.replication.computeIfAbsent(entry.getKey(), k -> { + ReplicatorStatsImpl r = new ReplicatorStatsImpl(); + r.setConnected(true); + return r; + }); + replStats.add(entry.getValue()); } + if (earliestMsgPublishTimeInBacklogs != 0 && ((TopicStatsImpl) ts).earliestMsgPublishTimeInBacklogs != 0) { earliestMsgPublishTimeInBacklogs = Math.min( earliestMsgPublishTimeInBacklogs, From c8183e741274cdaa1af4a9fe33c610f07ce274c8 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 14 Oct 2023 05:35:36 -0700 Subject: [PATCH 10/37] [feat] [broker] PIP-188 Add support to auto create topic resources into green cluster before migration [part-3] (#21354) --- conf/broker.conf | 4 + conf/standalone.conf | 8 ++ .../pulsar/broker/ServiceConfiguration.java | 7 + .../service/persistent/PersistentTopic.java | 123 ++++++++++++++++-- .../broker/service/ClusterMigrationTest.java | 88 ++++++++++++- .../common/policies/data/ClusterData.java | 5 +- .../apache/pulsar/admin/cli/CmdClusters.java | 9 +- .../policies/data/ClusterDataImplTest.java | 3 +- 8 files changed, 232 insertions(+), 15 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 4ad8536fd8d68..ca407810a42fc 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1593,6 +1593,10 @@ aggregatePublisherStatsByProducerName=false # if cluster is marked migrated. Disable with value 0. (Default disabled). clusterMigrationCheckDurationSeconds=0 +# Flag to start cluster migration for topic only after creating all topic's resources +# such as tenant, namespaces, subscriptions at new green cluster. (Default disabled). +clusterMigrationAutoResourceCreation=false + ### --- Schema storage --- ### # The schema storage implementation used by this broker schemaRegistryStorageClassName=org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorageFactory diff --git a/conf/standalone.conf b/conf/standalone.conf index 76223c5933e45..43455966c978f 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -999,6 +999,14 @@ splitTopicAndPartitionLabelInPrometheus=false # Otherwise, aggregate it by list index. aggregatePublisherStatsByProducerName=false +# Interval between checks to see if cluster is migrated and marks topic migrated +# if cluster is marked migrated. Disable with value 0. (Default disabled). +clusterMigrationCheckDurationSeconds=0 + +# Flag to start cluster migration for topic only after creating all topic's resources +# such as tenant, namespaces, subscriptions at new green cluster. (Default disabled). +clusterMigrationAutoResourceCreation=false + ### --- Schema storage --- ### # The schema storage implementation used by this broker. schemaRegistryStorageClassName=org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorageFactory 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 c93b683255002..82ddedd89ac0b 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 @@ -2778,6 +2778,13 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, ) private int clusterMigrationCheckDurationSeconds = 0; + @FieldContext( + category = CATEGORY_SERVER, + doc = "Flag to start cluster migration for topic only after creating all topic's resources" + + " such as tenant, namespaces, subscriptions at new green cluster. (Default disabled)." + ) + private boolean clusterMigrationAutoResourceCreation = false; + @FieldContext( category = CATEGORY_SCHEMA, doc = "Enforce schema validation on following cases:\n\n" 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 dfeb03a254698..1729ca878e7da 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 @@ -77,6 +77,7 @@ 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.net.BookieId; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -103,6 +104,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicClosedException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicFencedException; +import org.apache.pulsar.broker.service.BrokerServiceException.TopicMigratedException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException; import org.apache.pulsar.broker.service.BrokerServiceException.UnsupportedSubscriptionException; import org.apache.pulsar.broker.service.BrokerServiceException.UnsupportedVersionException; @@ -125,6 +127,8 @@ import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.OffloadProcessStatus; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException.ConflictException; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.BatchMessageIdImpl; @@ -137,10 +141,12 @@ import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.TxnAction; +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.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterData.ClusterUrl; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats; @@ -206,6 +212,9 @@ public static boolean isDedupCursorName(String name) { private static final long POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS = 60; + private static final String MIGRATION_CLUSTER_NAME = "migration-cluster"; + private volatile boolean migrationSubsCreated = false; + // topic has every published chunked message since topic is loaded public boolean msgChunkPublished; @@ -2582,16 +2591,110 @@ public CompletableFuture checkClusterMigration() { if (!clusterUrl.isPresent()) { return CompletableFuture.completedFuture(null); } - CompletableFuture migrated = !isMigrated() ? ledger.asyncMigrate() : - CompletableFuture.completedFuture(null); - return migrated.thenApply(__ -> { - subscriptions.forEach((name, sub) -> { - if (sub.isSubsciptionMigrated()) { - sub.getConsumers().forEach(Consumer::checkAndApplyTopicMigration); - } - }); - return null; - }).thenCompose(__ -> checkAndDisconnectReplicators()).thenCompose(__ -> checkAndUnsubscribeSubscriptions()); + return initMigration().thenCompose(subCreated -> { + migrationSubsCreated = true; + CompletableFuture migrated = !isMigrated() ? ledger.asyncMigrate() + : CompletableFuture.completedFuture(null); + return migrated.thenApply(__ -> { + subscriptions.forEach((name, sub) -> { + if (sub.isSubsciptionMigrated()) { + sub.getConsumers().forEach(Consumer::checkAndApplyTopicMigration); + } + }); + return null; + }).thenCompose(__ -> checkAndDisconnectReplicators()).thenCompose(__ -> checkAndUnsubscribeSubscriptions()); + }); + } + + /** + * Initialize migration for a topic by creating topic's resources at migration cluster. + */ + private CompletableFuture initMigration() { + if (migrationSubsCreated) { + return CompletableFuture.completedFuture(null); + } + log.info("{} initializing subscription created at migration cluster", topic); + return getMigratedClusterUrlAsync(getBrokerService().getPulsar()).thenCompose(clusterUrl -> { + if (!brokerService.getPulsar().getConfig().isClusterMigrationAutoResourceCreation()) { + return CompletableFuture.completedFuture(null); + } + if (!clusterUrl.isPresent()) { + return FutureUtil + .failedFuture(new TopicMigratedException("cluster migration service-url is not configired")); + } + ClusterUrl url = clusterUrl.get(); + ClusterData clusterData = ClusterData.builder().serviceUrl(url.getServiceUrl()) + .serviceUrlTls(url.getServiceUrlTls()).brokerServiceUrl(url.getBrokerServiceUrl()) + .brokerServiceUrlTls(url.getBrokerServiceUrlTls()).build(); + PulsarAdmin admin = getBrokerService().getClusterPulsarAdmin(MIGRATION_CLUSTER_NAME, + Optional.of(clusterData)); + + // namespace creation + final String tenant = TopicName.get(topic).getTenant(); + final NamespaceName ns = TopicName.get(topic).getNamespaceObject(); + List> subResults = new ArrayList<>(); + + return brokerService.getPulsar().getPulsarResources().getTenantResources().getTenantAsync(tenant) + .thenCompose(tenantInfo -> { + if (!tenantInfo.isPresent()) { + return CompletableFuture.completedFuture(null); + } + CompletableFuture ts = new CompletableFuture<>(); + admin.tenants().createTenantAsync(tenant, tenantInfo.get()).handle((__, ex) -> { + if (ex == null || ex instanceof ConflictException) { + log.info("[{}] successfully created tenant {} for migration", topic, tenant); + ts.complete(null); + return null; + } + log.warn("[{}] Failed to create tenant {} on migration cluster {}", topic, tenant, + ex.getCause().getMessage()); + ts.completeExceptionally(ex.getCause()); + return null; + }); + return ts; + }).thenCompose(t -> { + return brokerService.getPulsar().getPulsarResources().getNamespaceResources() + .getPoliciesAsync(ns).thenCompose(policies -> { + if (!policies.isPresent()) { + return CompletableFuture.completedFuture(null); + } + CompletableFuture nsFuture = new CompletableFuture<>(); + admin.namespaces().createNamespaceAsync(ns.toString(), policies.get()) + .handle((__, ex) -> { + if (ex == null || ex instanceof ConflictException) { + log.info("[{}] successfully created namespace {} for migration", + topic, ns); + nsFuture.complete(null); + return null; + } + log.warn("[{}] Failed to create namespace {} on migration cluster {}", + topic, ns, ex.getCause().getMessage()); + nsFuture.completeExceptionally(ex.getCause()); + return null; + }); + return nsFuture; + }).thenCompose(p -> { + subscriptions.forEach((subName, sub) -> { + CompletableFuture subResult = new CompletableFuture<>(); + subResults.add(subResult); + admin.topics().createSubscriptionAsync(topic, subName, MessageId.earliest) + .handle((__, ex) -> { + if (ex == null || ex instanceof ConflictException) { + log.info("[{}] successfully created sub {} for migration", + topic, subName); + subResult.complete(null); + return null; + } + log.warn("[{}] Failed to create sub {} on migration cluster, {}", + topic, subName, ex.getCause().getMessage()); + subResult.completeExceptionally(ex.getCause()); + return null; + }); + }); + return Futures.waitForAll(subResults); + }); + }); + }); } private CompletableFuture checkAndUnsubscribeSubscriptions() { 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 248d87c631c4c..ec1cb7e4fc9e8 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 @@ -28,15 +28,18 @@ import java.lang.reflect.Method; import java.net.URL; +import java.util.List; import java.util.Optional; import java.util.concurrent.TimeUnit; 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.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.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -267,7 +270,8 @@ public void testClusterMigration(boolean persistent, SubscriptionType subType) t AbstractTopic topic2 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName, false).getNow(null).get(); assertFalse(topic2.getProducers().isEmpty()); - ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), + pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); admin1.clusters().updateClusterMigration("r1", true, migratedUrl); retryStrategically((test) -> { @@ -454,7 +458,8 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc retryStrategically((test) -> topic2.getReplicators().size() == 1, 10, 2000); log.info("replicators should be ready"); - ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), + pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); admin1.clusters().updateClusterMigration("r1", true, migratedUrl); log.info("update cluster migration called"); retryStrategically((test) -> { @@ -494,6 +499,85 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc assertEquals(topic2.getProducers().size(), 2); } + /** + * This test validates that blue cluster first creates list of subscriptions into green cluster so, green cluster + * will not lose the data if producer migrates. + * + * @throws Exception + */ + @Test + public void testClusterMigrationWithResourceCreated() throws Exception { + log.info("--- Starting testClusterMigrationWithResourceCreated ---"); + + String tenant = "pulsar2"; + String namespace = tenant + "/migration"; + String greenClusterName = pulsar2.getConfig().getClusterName(); + String blueClusterName = pulsar1.getConfig().getClusterName(); + admin1.clusters().createCluster(greenClusterName, + ClusterData.builder().serviceUrl(url2.toString()).serviceUrlTls(urlTls2.toString()) + .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()).build()); + admin2.clusters().createCluster(blueClusterName, + ClusterData.builder().serviceUrl(url1.toString()).serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()).build()); + + admin1.tenants().createTenant(tenant, new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), + Sets.newHashSet("r1", greenClusterName))); + // broker should handle already tenant creation + admin2.tenants().createTenant(tenant, new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), + Sets.newHashSet("r1", greenClusterName))); + admin1.namespaces().createNamespace(namespace, Sets.newHashSet("r1", greenClusterName)); + + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/migrationTopic"); + + broker1.getPulsarService().getConfig().setClusterMigrationAutoResourceCreation(true); + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + // cluster-1 producer/consumer + Producer producer1 = client1.newProducer().topic(topicName).enableBatching(false) + .producerName("cluster1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + // create subscriptions + admin1.topics().createSubscription(topicName, "s1", MessageId.earliest); + admin1.topics().createSubscription(topicName, "s2", MessageId.earliest); + + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), + pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); + admin1.clusters().updateClusterMigration("r1", true, migratedUrl); + + PersistentTopic topic1 = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).getNow(null) + .get(); + retryStrategically((test) -> { + try { + topic1.checkClusterMigration().get(); + return true; + } catch (Exception e) { + // ok + } + return false; + }, 10, 500); + + assertNotNull(admin2.tenants().getTenantInfo(tenant)); + assertNotNull(admin2.namespaces().getPolicies(namespace)); + List subLists = admin2.topics().getSubscriptions(topicName); + assertTrue(subLists.contains("s1")); + assertTrue(subLists.contains("s2")); + + int n = 5; + for (int i = 0; i < n; i++) { + producer1.send("test1".getBytes()); + } + + Consumer consumer1 = client1.newConsumer().topic(topicName).subscriptionName("s1").subscribe(); + for (int i = 0; i < n; i++) { + assertNotNull(consumer1.receive()); + } + + consumer1.close(); + producer1.close(); + } + static class TestBroker extends MockedPulsarServiceBaseTest { private String clusterName; 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 212a1575f9934..0b3e5aa49cb83 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 @@ -136,11 +136,14 @@ static Builder builder() { @NoArgsConstructor @AllArgsConstructor class ClusterUrl { + String serviceUrl; + String serviceUrlTls; String brokerServiceUrl; String brokerServiceUrlTls; public boolean isEmpty() { - return brokerServiceUrl == null && brokerServiceUrlTls == null; + return serviceUrl != null && serviceUrlTls != null && brokerServiceUrl == null + && brokerServiceUrlTls == null; } } } 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 1653de93a738c..c578876b382fb 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 @@ -162,6 +162,13 @@ private class UpdateClusterMigration extends CliCommand { @Parameter(names = "--migrated", description = "Is cluster migrated", required = true) private boolean migrated; + @Parameter(names = "--service-url", description = "New migrated cluster service url", required = false) + private String serviceUrl; + + @Parameter(names = "--service-url-secure", + description = "New migrated cluster service url secure", required = false) + private String serviceUrlTls; + @Parameter(names = "--broker-url", description = "New migrated cluster broker service url", required = false) private String brokerServiceUrl; @@ -171,7 +178,7 @@ private class UpdateClusterMigration extends CliCommand { void run() throws PulsarAdminException { String cluster = getOneArgument(params); - ClusterUrl clusterUrl = new ClusterUrl(brokerServiceUrl, brokerServiceUrlTls); + ClusterUrl clusterUrl = new ClusterUrl(serviceUrl, serviceUrlTls, brokerServiceUrl, brokerServiceUrlTls); getAdmin().clusters().updateClusterMigration(cluster, migrated, clusterUrl); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataImplTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataImplTest.java index ca4cba2cf9749..87e935ecf7360 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataImplTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/ClusterDataImplTest.java @@ -54,7 +54,8 @@ public void verifyClone() { .brokerClientCertificateFilePath("/my/cert/file") .listenerName("a-listener") .migrated(true) - .migratedClusterUrl(new ClusterData.ClusterUrl("pulsar://remote", "pulsar+ssl://remote")) + .migratedClusterUrl(new ClusterData.ClusterUrl("http://remote", "https://remote", "pulsar://remote", + "pulsar+ssl://remote")) .build(); ClusterDataImpl clone = originalData.clone().build(); From a5f4c1e0e612e7568bb4b6be1e41f71bc4ac4ff2 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 14 Oct 2023 06:43:38 -0700 Subject: [PATCH 11/37] [fix][broker] make closing producer thread-safe while updating recently closed producer (#21355) --- .../java/org/apache/pulsar/broker/service/ServerCnx.java | 7 +++---- 1 file changed, 3 insertions(+), 4 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 0517fff0f03f5..34542d56938b0 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 @@ -46,7 +46,6 @@ 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; @@ -56,6 +55,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; @@ -186,7 +186,7 @@ public class ServerCnx extends PulsarHandler implements TransportCnx { private final BrokerService service; private final SchemaRegistryService schemaService; private final String listenerName; - private final HashMap recentlyClosedProducers; + private final Map recentlyClosedProducers; private final ConcurrentLongHashMap> producers; private final ConcurrentLongHashMap> consumers; private final boolean enableSubscriptionPatternEvaluation; @@ -291,7 +291,7 @@ public ServerCnx(PulsarService pulsar, String listenerName) { .expectedItems(8) .concurrencyLevel(1) .build(); - this.recentlyClosedProducers = new HashMap<>(); + this.recentlyClosedProducers = new ConcurrentHashMap<>(); this.replicatorPrefix = conf.getReplicatorPrefix(); this.maxNonPersistentPendingMessages = conf.getMaxConcurrentNonPersistentMessagePerConnection(); this.schemaValidationEnforced = conf.isSchemaValidationEnforced(); @@ -2984,7 +2984,6 @@ protected void interceptCommand(BaseCommand command) throws InterceptException { @Override public void closeProducer(Producer producer) { - assert ctx.executor().inEventLoop(); // removes producer-connection from map and send close command to producer safelyRemoveProducer(producer); if (getRemoteEndpointProtocolVersion() >= v5.getValue()) { From d09642c7cbfc18cf532aaebf550f5ac6206c5c4b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 14 Oct 2023 07:34:08 -0700 Subject: [PATCH 12/37] [fix][client] fix producer/consumer perform lookup for migrated topic (#21356) Co-authored-by: Rajan Dhabalia --- .../pulsar/client/impl/ConnectionHandler.java | 13 +++++++-- .../pulsar/client/impl/LookupService.java | 1 - .../pulsar/client/impl/PulsarClientImpl.java | 29 +++++++++++++++++-- 3 files changed, 36 insertions(+), 7 deletions(-) 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 fc7c89c3ce693..6979914274e85 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 @@ -88,9 +88,16 @@ protected void grabCnx() { try { CompletableFuture cnxFuture; if (state.redirectedClusterURI != null) { - InetSocketAddress address = InetSocketAddress.createUnresolved(state.redirectedClusterURI.getHost(), - state.redirectedClusterURI.getPort()); - cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); + if (state.topic == null) { + InetSocketAddress address = InetSocketAddress.createUnresolved(state.redirectedClusterURI.getHost(), + state.redirectedClusterURI.getPort()); + cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); + } else { + // once, client receives redirection url, client has to perform lookup on migrated + // cluster to find the broker that owns the topic and then create connection. + // below method, performs the lookup for a given topic and then creates connection + cnxFuture = state.client.getConnection(state.topic, (state.redirectedClusterURI.toString())); + } } else if (state.topic == null) { cnxFuture = state.client.getConnectionToServiceUrl(); } else { 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 48ef67eae2047..f0142f3612b4c 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 @@ -105,5 +105,4 @@ public interface LookupService extends AutoCloseable { */ CompletableFuture getTopicsUnderNamespace(NamespaceName namespace, Mode mode, String topicPattern, String topicsHash); - } 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 fdabb5fa8cfa5..50a3dbfc935b7 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.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -104,6 +105,7 @@ public class PulsarClientImpl implements PulsarClient { private final boolean createdScheduledProviders; private LookupService lookup; + private Map urlLookupMap = new ConcurrentHashMap<>(); private final ConnectionPool cnxPool; @Getter private final Timer timer; @@ -960,6 +962,23 @@ public CompletableFuture getConnection(final String topic) { .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); } + public CompletableFuture getConnection(final String topic, final String url) { + TopicName topicName = TopicName.get(topic); + return getLookup(url).getBroker(topicName) + .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); + } + + public LookupService getLookup(String serviceUrl) { + return urlLookupMap.computeIfAbsent(serviceUrl, url -> { + try { + return createLookup(serviceUrl); + } catch (PulsarClientException e) { + log.warn("Failed to update url to lookup service {}, {}", url, e.getMessage()); + throw new IllegalStateException("Failed to update url " + url); + } + }); + } + public CompletableFuture getConnectionToServiceUrl() { if (!(lookup instanceof BinaryProtoLookupService)) { return FutureUtil.failedFuture(new PulsarClientException.InvalidServiceURL( @@ -1018,10 +1037,14 @@ public LookupService getLookup() { } public void reloadLookUp() throws PulsarClientException { - if (conf.getServiceUrl().startsWith("http")) { - lookup = new HttpLookupService(conf, eventLoopGroup); + lookup = createLookup(conf.getServiceUrl()); + } + + public LookupService createLookup(String url) throws PulsarClientException { + if (url.startsWith("http")) { + return new HttpLookupService(conf, eventLoopGroup); } else { - lookup = new BinaryProtoLookupService(this, conf.getServiceUrl(), conf.getListenerName(), conf.isUseTls(), + return new BinaryProtoLookupService(this, url, conf.getListenerName(), conf.isUseTls(), externalExecutorProvider.getExecutor()); } } From a10564d875f42c32fda0178cfe7ad976d26fbfb1 Mon Sep 17 00:00:00 2001 From: vraulji567 <95091480+vraulji567@users.noreply.github.com> Date: Sun, 15 Oct 2023 20:32:42 -0400 Subject: [PATCH 13/37] [fix][broker]Support to migrate topics from blue to green cluster per namespace (#21367) Co-authored-by: Vishwadeepsinh Raulji --- .../broker/admin/impl/NamespacesBase.java | 14 + .../pulsar/broker/admin/v1/Namespaces.java | 13 + .../pulsar/broker/admin/v2/Namespaces.java | 12 +- .../pulsar/broker/service/AbstractTopic.java | 22 +- .../pulsar/broker/service/Consumer.java | 3 +- .../pulsar/broker/service/ServerCnx.java | 2 +- .../nonpersistent/NonPersistentTopic.java | 11 +- .../service/persistent/PersistentTopic.java | 5 +- .../broker/service/ClusterMigrationTest.java | 455 ++++++++++++++++++ .../pulsar/client/admin/Namespaces.java | 27 +- .../pulsar/common/policies/data/Policies.java | 2 + .../client/admin/internal/NamespacesImpl.java | 11 + .../apache/pulsar/admin/cli/CmdClusters.java | 11 +- .../pulsar/admin/cli/CmdNamespaces.java | 16 + 14 files changed, 579 insertions(+), 25 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 8ab1f4dc86002..a8f1af1d34f91 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 @@ -2642,4 +2642,18 @@ protected void internalRemoveBacklogQuota(AsyncResponse asyncResponse, BacklogQu return null; }); } + + protected void internalEnableMigration(boolean migrated) { + validateSuperUserAccess(); + try { + updatePolicies(namespaceName, policies -> { + policies.isMigrated = migrated; + return policies; + }); + log.info("Successfully updated migration on namespace {}", namespaceName); + } catch (Exception e) { + log.error("Failed to update migration on namespace {}", namespaceName, e); + throw new RestException(e); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java index 234d77251139b..b188750252ab3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java @@ -1709,5 +1709,18 @@ public void setSchemaAutoUpdateCompatibilityStrategy(@PathParam("tenant") String internalSetSchemaAutoUpdateCompatibilityStrategy(strategy); } + @POST + @Path("/{property}/{cluster}/{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"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void enableMigration(@PathParam("property") String property, + @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, + boolean migrated) { + validateNamespaceName(property, cluster, namespace); + internalEnableMigration(migrated); + } + private static final Logger log = LoggerFactory.getLogger(Namespaces.class); } 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 dfa040baec5cf..36df0f7e31a34 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 @@ -2779,7 +2779,17 @@ 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"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void enableMigration(@PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + boolean migrated) { + validateNamespaceName(tenant, namespace); + internalEnableMigration(migrated); + } private static final Logger log = LoggerFactory.getLogger(Namespaces.class); } 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 31e37d0f176d7..3cb396d7a4b41 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 @@ -1349,19 +1349,29 @@ public void updateBrokerSubscribeRate() { } public Optional getMigratedClusterUrl() { - return getMigratedClusterUrl(brokerService.getPulsar()); + return getMigratedClusterUrl(brokerService.getPulsar(), topic); } - public static CompletableFuture> getMigratedClusterUrlAsync(PulsarService pulsar) { + public static CompletableFuture> getMigratedClusterUrlAsync(PulsarService pulsar, + String topic) { return pulsar.getPulsarResources().getClusterResources().getClusterAsync(pulsar.getConfig().getClusterName()) - .thenApply(clusterData -> (clusterData.isPresent() && clusterData.get().isMigrated()) + .thenCombine(isNamespaceMigrationEnabledAsync(pulsar, topic), + ((clusterData, isNamespaceMigrationEnabled) + -> ((clusterData.isPresent() && clusterData.get().isMigrated()) + || isNamespaceMigrationEnabled) ? Optional.ofNullable(clusterData.get().getMigratedClusterUrl()) - : Optional.empty()); + : Optional.empty())); } - public static Optional getMigratedClusterUrl(PulsarService pulsar) { + private static CompletableFuture isNamespaceMigrationEnabledAsync(PulsarService pulsar, String topic) { + return pulsar.getPulsarResources().getNamespaceResources(). + getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) + .thenApply(policies -> policies.isPresent() && policies.get().isMigrated); + } + + public static Optional getMigratedClusterUrl(PulsarService pulsar, String topic) { try { - return getMigratedClusterUrlAsync(pulsar) + return getMigratedClusterUrlAsync(pulsar, topic) .get(pulsar.getPulsarResources().getClusterResources().getOperationTimeoutSec(), TimeUnit.SECONDS); } catch (Exception e) { log.warn("Failed to get migration cluster URL", e); 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 0ca53eac2f4d7..68678efc29637 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 @@ -821,7 +821,8 @@ public void topicMigrated(Optional clusterUrl) { public boolean checkAndApplyTopicMigration() { if (subscription.isSubsciptionMigrated()) { - Optional clusterUrl = AbstractTopic.getMigratedClusterUrl(cnx.getBrokerService().getPulsar()); + Optional clusterUrl = AbstractTopic.getMigratedClusterUrl(cnx.getBrokerService().getPulsar(), + topicName); if (clusterUrl.isPresent()) { ClusterUrl url = clusterUrl.get(); cnx.getCommandSender().sendTopicMigrated(ResourceType.Consumer, consumerId, url.getBrokerServiceUrl(), 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 34542d56938b0..18bf62f1ef34a 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 @@ -1644,7 +1644,7 @@ private void buildProducerAndAddTopic(Topic topic, long producerId, String produ producers.remove(producerId, producerFuture); }).exceptionallyAsync(ex -> { if (ex.getCause() instanceof BrokerServiceException.TopicMigratedException) { - Optional clusterURL = getMigratedClusterUrl(service.getPulsar()); + Optional clusterURL = getMigratedClusterUrl(service.getPulsar(), topic.getName()); if (clusterURL.isPresent()) { if (topic.isReplicationBacklogExist()) { log.info("Topic {} is migrated but replication backlog exist: " 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 639d2cfc5810f..54811da723808 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 @@ -156,11 +156,8 @@ public NonPersistentTopic(String topic, BrokerService brokerService) { } private CompletableFuture updateClusterMigrated() { - return getMigratedClusterUrlAsync(brokerService.getPulsar()).thenAccept(url -> migrated = url.isPresent()); - } - - private Optional getClusterMigrationUrl() { - return getMigratedClusterUrl(brokerService.getPulsar()); + return getMigratedClusterUrlAsync(brokerService.getPulsar(), topic) + .thenAccept(url -> migrated = url.isPresent()); } public CompletableFuture initialize() { @@ -332,7 +329,7 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St false, cnx, cnx.getAuthRole(), metadata, readCompacted, keySharedMeta, MessageId.latest, DEFAULT_CONSUMER_EPOCH, schemaType); if (isMigrated()) { - consumer.topicMigrated(getClusterMigrationUrl()); + consumer.topicMigrated(getMigratedClusterUrl()); } addConsumerToSubscription(subscription, consumer).thenRun(() -> { @@ -949,7 +946,7 @@ public boolean isActive() { @Override public CompletableFuture checkClusterMigration() { - Optional url = getClusterMigrationUrl(); + Optional url = getMigratedClusterUrl(); if (url.isPresent()) { this.migrated = true; producers.forEach((__, 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 1729ca878e7da..1ed7bafc14724 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 @@ -2588,6 +2588,7 @@ private boolean hasBacklogs() { @Override public CompletableFuture checkClusterMigration() { Optional clusterUrl = getMigratedClusterUrl(); + if (!clusterUrl.isPresent()) { return CompletableFuture.completedFuture(null); } @@ -2614,13 +2615,13 @@ private CompletableFuture initMigration() { return CompletableFuture.completedFuture(null); } log.info("{} initializing subscription created at migration cluster", topic); - return getMigratedClusterUrlAsync(getBrokerService().getPulsar()).thenCompose(clusterUrl -> { + return getMigratedClusterUrlAsync(getBrokerService().getPulsar(), topic).thenCompose(clusterUrl -> { if (!brokerService.getPulsar().getConfig().isClusterMigrationAutoResourceCreation()) { return CompletableFuture.completedFuture(null); } if (!clusterUrl.isPresent()) { return FutureUtil - .failedFuture(new TopicMigratedException("cluster migration service-url is not configired")); + .failedFuture(new TopicMigratedException("cluster migration service-url is not configured")); } ClusterUrl url = clusterUrl.get(); ClusterData clusterData = ClusterData.builder().serviceUrl(url.getServiceUrl()) 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 ec1cb7e4fc9e8..b1fd11ba0ac42 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 @@ -66,6 +66,7 @@ public class ClusterMigrationTest { protected String methodName; String namespace = "pulsar/migrationNs"; + String namespaceNotToMigrate = "pulsar/notToMigrateNs"; TestBroker broker1, broker2, broker3, broker4; URL url1; @@ -104,6 +105,24 @@ public Object[][] subscriptionTypes() { }; } + @DataProvider(name="NamespaceMigrationTopicSubscriptionTypes") + public Object[][] namespaceMigrationSubscriptionTypes() { + return new Object[][] { + {true, SubscriptionType.Shared, true, false}, + {true, SubscriptionType.Key_Shared, true, false}, + {true, SubscriptionType.Shared, false, true}, + {true, SubscriptionType.Key_Shared, false, true}, + {true, SubscriptionType.Shared, true, true}, + {true, SubscriptionType.Key_Shared, true, true}, + {false, SubscriptionType.Shared, true, false}, + {false, SubscriptionType.Key_Shared, true, false}, + {false, SubscriptionType.Shared, false, true}, + {false, SubscriptionType.Key_Shared,false, true}, + {false, SubscriptionType.Shared, true, true}, + {false, SubscriptionType.Key_Shared,true, true}, + }; + } + @BeforeMethod(alwaysRun = true, timeOut = 300000) public void setup() throws Exception { @@ -179,6 +198,9 @@ public void setup() throws Exception { admin1.namespaces().createNamespace(namespace, Sets.newHashSet("r1", "r3")); admin3.namespaces().createNamespace(namespace); admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r3")); + admin1.namespaces().createNamespace(namespaceNotToMigrate, Sets.newHashSet("r1", "r3")); + admin3.namespaces().createNamespace(namespaceNotToMigrate); + admin1.namespaces().setNamespaceReplicationClusters(namespaceNotToMigrate, Sets.newHashSet("r1", "r3")); // Setting r4 as replication cluster for r2 admin2.tenants().createTenant("pulsar", @@ -188,6 +210,9 @@ public void setup() throws Exception { admin2.namespaces().createNamespace(namespace, Sets.newHashSet("r2", "r4")); admin4.namespaces().createNamespace(namespace); admin2.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r2", "r4")); + admin2.namespaces().createNamespace(namespaceNotToMigrate, Sets.newHashSet("r2", "r4")); + admin4.namespaces().createNamespace(namespaceNotToMigrate); + admin2.namespaces().setNamespaceReplicationClusters(namespaceNotToMigrate, Sets.newHashSet("r2", "r4")); assertEquals(admin1.clusters().getCluster("r1").getServiceUrl(), url1.toString()); assertEquals(admin2.clusters().getCluster("r2").getServiceUrl(), url2.toString()); @@ -458,10 +483,12 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc retryStrategically((test) -> topic2.getReplicators().size() == 1, 10, 2000); log.info("replicators should be ready"); + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); admin1.clusters().updateClusterMigration("r1", true, migratedUrl); log.info("update cluster migration called"); + retryStrategically((test) -> { try { topic1.checkClusterMigration().get(); @@ -578,6 +605,434 @@ public void testClusterMigrationWithResourceCreated() throws Exception { producer1.close(); } + @Test(dataProvider = "NamespaceMigrationTopicSubscriptionTypes") + public void testNamespaceMigration(boolean persistent, SubscriptionType subType, boolean isClusterMigrate, boolean isNamespaceMigrate) throws Exception { + log.info("--- Starting Test::testNamespaceMigration ---"); + // topic for the namespace1 (to be migrated) + final String topicName = BrokerTestUtil + .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); + // topic for namespace2 (not to be migrated) + final String topicName2 = BrokerTestUtil + .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespaceNotToMigrate + "/migrationTopic"); + + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + // blue cluster - namespace1 - producer/consumer + Producer blueProducerNs1_1 = client1.newProducer().topic(topicName).enableBatching(false) + .producerName("blue-producer-ns1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + Consumer blueConsumerNs1_1 = client1.newConsumer().topic(topicName).subscriptionType(subType) + .subscriptionName("s1").subscribe(); + AbstractTopic blueTopicNs1_1 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName, false).getNow(null).get(); + retryStrategically((test) -> !blueTopicNs1_1.getProducers().isEmpty(), 5, 500); + retryStrategically((test) -> !blueTopicNs1_1.getSubscriptions().isEmpty(), 5, 500); + assertFalse(blueTopicNs1_1.getProducers().isEmpty()); + assertFalse(blueTopicNs1_1.getSubscriptions().isEmpty()); + + // blue cluster - namespace2 - producer/consumer + Producer blueProducerNs2_1 = client1.newProducer().topic(topicName2).enableBatching(false) + .producerName("blue-producer-ns2-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + Consumer blueConsumerNs2_1 = client1.newConsumer().topic(topicName2).subscriptionType(subType) + .subscriptionName("s1").subscribe(); + AbstractTopic blueTopicNs2_1 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName2, false).getNow(null).get(); + retryStrategically((test) -> !blueTopicNs2_1.getProducers().isEmpty(), 5, 500); + retryStrategically((test) -> !blueTopicNs2_1.getSubscriptions().isEmpty(), 5, 500); + assertFalse(blueTopicNs2_1.getProducers().isEmpty()); + assertFalse(blueTopicNs2_1.getSubscriptions().isEmpty()); + + // build backlog on the blue cluster + blueConsumerNs1_1.close(); + blueConsumerNs2_1.close(); + int n = 5; + for (int i = 0; i < n; i++) { + blueProducerNs1_1.send("test1".getBytes()); + blueProducerNs2_1.send("test1".getBytes()); + } + + @Cleanup + PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + // green cluster - namespace1 - producer/consumer + Producer greenProducerNs1_1 = client2.newProducer().topic(topicName).enableBatching(false) + .producerName("green-producer-ns1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + AbstractTopic greenTopicNs1_1 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName, false).getNow(null).get(); + assertFalse(greenTopicNs1_1.getProducers().isEmpty()); + + // green cluster - namespace2 - producer/consumer + Producer greenProducerNs2_1 = client2.newProducer().topic(topicName2).enableBatching(false) + .producerName("cluster2-nm1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + AbstractTopic greenTopicNs2_1 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName2, false).getNow(null).get(); + assertFalse(greenTopicNs2_1.getProducers().isEmpty()); + + // blue - green cluster migration + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), + pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); + admin1.clusters().updateClusterMigration("r1", isClusterMigrate, migratedUrl); + admin1.namespaces().updateMigrationState(namespace, isNamespaceMigrate); + + retryStrategically((test) -> { + try { + blueTopicNs1_1.checkClusterMigration().get(); + if (isClusterMigrate) { + blueTopicNs2_1.checkClusterMigration().get(); + } + return true; + } catch (Exception e) { + // ok + } + return false; + }, 10, 500); + + + blueTopicNs1_1.checkClusterMigration().get(); + if (isClusterMigrate) { + blueTopicNs2_1.checkClusterMigration().get(); + } + + log.info("before sending message"); + sleep(1000); + blueProducerNs1_1.sendAsync("test1".getBytes()); + blueProducerNs2_1.sendAsync("test1".getBytes()); + + // producer is disconnected from blue for namespace1 as cluster or namespace migration is enabled + retryStrategically((test) -> blueTopicNs1_1.getProducers().isEmpty(), 10, 500); + assertTrue(blueTopicNs1_1.getProducers().isEmpty()); + + if(isClusterMigrate){ + // producer is disconnected from blue for namespace2 if cluster migration is enabled + retryStrategically((test) -> blueTopicNs2_1.getProducers().isEmpty(), 10, 500); + assertTrue(blueTopicNs2_1.getProducers().isEmpty()); + } else { + // producer is not disconnected from blue for namespace2 if namespace migration is disabled + retryStrategically((test) -> !blueTopicNs2_1.getProducers().isEmpty(), 10, 500); + assertTrue(!blueTopicNs2_1.getProducers().isEmpty()); + } + + // create producer on blue which should be redirected to green + Producer blueProducerNs1_2 = client1.newProducer().topic(topicName).enableBatching(false) + .producerName("blue-producer-ns1-2").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + + // producer is connected with green + retryStrategically((test) -> greenTopicNs1_1.getProducers().size() == 3, 10, 500); + assertTrue(greenTopicNs1_1.getProducers().size() == 3); + + // blueProducerNs2_1 should be migrated to green if the cluster migration is enabled + // should not be migrated if the namespace migration is disabled for namespace2 + if (isClusterMigrate) { + retryStrategically((test) -> greenTopicNs2_1.getProducers().size() == 2, 10, 500); + assertTrue(greenTopicNs2_1.getProducers().size() == 2); + } else{ + retryStrategically((test) -> greenTopicNs2_1.getProducers().size() == 1, 10, 500); + assertTrue(greenTopicNs2_1.getProducers().size() == 1); + } + + // try to consume backlog messages from cluster-1 + blueConsumerNs1_1 = client1.newConsumer().topic(topicName).subscriptionName("s1").subscribe(); + blueConsumerNs2_1 = client1.newConsumer().topic(topicName2).subscriptionName("s1").subscribe(); + if (persistent) { + for (int i = 0; i < n; i++) { + Message msg = blueConsumerNs1_1.receive(); + assertEquals(msg.getData(), "test1".getBytes()); + blueConsumerNs1_1.acknowledge(msg); + + Message msg2 = blueConsumerNs2_1.receive(); + assertEquals(msg2.getData(), "test1".getBytes()); + blueConsumerNs2_1.acknowledge(msg2); + } + } + // after consuming all messages, consumer should have disconnected + // from blue and reconnect with green + retryStrategically((test) -> !greenTopicNs1_1.getSubscriptions().isEmpty(), 10, 500); + assertFalse(greenTopicNs1_1.getSubscriptions().isEmpty()); + if (isClusterMigrate) { + retryStrategically((test) -> !greenTopicNs2_1.getSubscriptions().isEmpty(), 10, 500); + assertFalse(greenTopicNs2_1.getSubscriptions().isEmpty()); + } else { + retryStrategically((test) -> greenTopicNs2_1.getSubscriptions().isEmpty(), 10, 500); + assertTrue(greenTopicNs2_1.getSubscriptions().isEmpty()); + } + + blueTopicNs1_1.checkClusterMigration().get(); + if (isClusterMigrate) { + blueTopicNs2_1.checkClusterMigration().get(); + } + + ConcurrentOpenHashMap replicators = blueTopicNs1_1.getReplicators(); + replicators.forEach((r, replicator) -> { + assertFalse(replicator.isConnected()); + }); + assertTrue(blueTopicNs1_1.getSubscriptions().isEmpty()); + + if (isClusterMigrate) { + ConcurrentOpenHashMap replicatorsNm = blueTopicNs2_1.getReplicators(); + replicatorsNm.forEach((r, replicator) -> { + assertFalse(replicator.isConnected()); + }); + assertTrue(blueTopicNs2_1.getSubscriptions().isEmpty()); + } else { + ConcurrentOpenHashMap replicatorsNm = blueTopicNs2_1.getReplicators(); + replicatorsNm.forEach((r, replicator) -> { + assertTrue(replicator.isConnected()); + }); + assertFalse(blueTopicNs2_1.getSubscriptions().isEmpty()); + } + + // create a new consumer on blue which should also reconnect to green + Consumer blueConsumerNs1_2 = client1.newConsumer().topic(topicName).subscriptionType(subType) + .subscriptionName("s2").subscribe(); + Consumer blueConsumerNs2_2 = client1.newConsumer().topic(topicName2).subscriptionType(subType) + .subscriptionName("s2").subscribe(); + retryStrategically((test) -> greenTopicNs1_1.getSubscription("s2") != null, 10, 500); + assertFalse(greenTopicNs1_1.getSubscription("s2").getConsumers().isEmpty()); + if (isClusterMigrate) { + retryStrategically((test) -> greenTopicNs2_1.getSubscription("s2") != null, 10, 500); + assertFalse(greenTopicNs2_1.getSubscription("s2").getConsumers().isEmpty()); + } else { + retryStrategically((test) -> greenTopicNs2_1.getSubscription("s2") == null, 10, 500); + } + + // new sub on migration topic must be redirected immediately + Consumer consumerM = client1.newConsumer().topic(topicName).subscriptionType(subType) + .subscriptionName("sM").subscribe(); + assertFalse(pulsar2.getBrokerService().getTopicReference(topicName).get().getSubscription("sM").getConsumers() + .isEmpty()); + consumerM.close(); + + // migrate topic after creating subscription + String newTopicName = topicName + "-new"; + consumerM = client1.newConsumer().topic(newTopicName).subscriptionType(subType) + .subscriptionName("sM").subscribe(); + retryStrategically((t) -> pulsar1.getBrokerService().getTopicReference(newTopicName).isPresent(), 5, 100); + pulsar1.getBrokerService().getTopicReference(newTopicName).get().checkClusterMigration().get(); + retryStrategically((t) -> + pulsar2.getBrokerService().getTopicReference(newTopicName).isPresent() && + pulsar2.getBrokerService().getTopicReference(newTopicName).get().getSubscription("sM") + .getConsumers().isEmpty(), 5, 100); + assertFalse(pulsar2.getBrokerService().getTopicReference(newTopicName).get().getSubscription("sM").getConsumers() + .isEmpty()); + consumerM.close(); + + // publish messages to cluster-2 and consume them + for (int i = 0; i < n; i++) { + blueProducerNs1_1.send("test2".getBytes()); + blueProducerNs1_2.send("test2".getBytes()); + greenProducerNs1_1.send("test2".getBytes()); + } + log.info("Successfully published messages by migrated producers"); + for (int i = 0; i < n * 3; i++) { + assertEquals(blueConsumerNs1_1.receive(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); + assertEquals(blueConsumerNs1_2.receive(2, TimeUnit.SECONDS).getData(), "test2".getBytes()); + + } + + // create non-migrated topic which should connect to blue + String diffTopic = BrokerTestUtil + .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); + Consumer consumerDiff = client1.newConsumer().topic(diffTopic).subscriptionType(subType) + .subscriptionName("s1-d").subscribe(); + Producer producerDiff = client1.newProducer().topic(diffTopic).enableBatching(false) + .producerName("cluster1-d").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + AbstractTopic topicDiff = (AbstractTopic) pulsar1.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); + assertNotNull(topicDiff); + for (int i = 0; i < n; i++) { + producerDiff.send("diff".getBytes()); + assertEquals(consumerDiff.receive(2, TimeUnit.SECONDS).getData(), "diff".getBytes()); + } + + // restart broker-1 + broker1.restart(); + Producer producer4 = client1.newProducer().topic(topicName).enableBatching(false) + .producerName("cluster1-4").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + Consumer consumer3 = client1.newConsumer().topic(topicName).subscriptionType(subType) + .subscriptionName("s3").subscribe(); + retryStrategically((test) -> greenTopicNs1_1.getProducers().size() == 4, 10, 500); + assertTrue(greenTopicNs1_1.getProducers().size() == 4); + retryStrategically((test) -> greenTopicNs1_1.getSubscription("s3") != null, 10, 500); + assertFalse(greenTopicNs1_1.getSubscription("s3").getConsumers().isEmpty()); + for (int i = 0; i < n; i++) { + producer4.send("test3".getBytes()); + assertEquals(blueConsumerNs1_1.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); + assertEquals(blueConsumerNs1_2.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); + assertEquals(consumer3.receive(2, TimeUnit.SECONDS).getData(), "test3".getBytes()); + } + + log.info("Successfully consumed messages by migrated consumers"); + + // clean up + blueConsumerNs1_1.close(); + blueConsumerNs1_2.close(); + blueConsumerNs2_1.close(); + blueProducerNs1_1.close(); + blueProducerNs1_2.close(); + blueProducerNs2_1.close(); + greenProducerNs1_1.close(); + greenProducerNs2_1.close(); + } + + @Test(dataProvider = "NamespaceMigrationTopicSubscriptionTypes") + public void testNamespaceMigrationWithReplicationBacklog(boolean persistent, SubscriptionType subType, boolean isClusterMigrate, boolean isNamespaceMigrate) throws Exception { + log.info("--- Starting ReplicatorTest::testNamespaceMigrationWithReplicationBacklog ---"); + persistent = true; + // topic for namespace1 (to be migrated) + final String topicName = BrokerTestUtil + .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespace + "/migrationTopic"); + // topic for namespace2 (not to be migrated) + final String topicName2 = BrokerTestUtil + .newUniqueName((persistent ? "persistent" : "non-persistent") + "://" + namespaceNotToMigrate + "/migrationTopic"); + + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + @Cleanup + PulsarClient client3 = PulsarClient.builder().serviceUrl(url3.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + // blue cluster - namespace1 - producer/consumer + Producer blueProducerNs1_1 = client1.newProducer().topic(topicName).enableBatching(false) + .producerName("blue-producer-ns1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + Consumer blueConsumerNs1_1 = client1.newConsumer().topic(topicName).subscriptionType(subType) + .subscriptionName("s1").subscribe(); + + // blue cluster - namespace2 - producer/consumer + Producer blueProducerNs2_1 = client1.newProducer().topic(topicName2).enableBatching(false) + .producerName("blue-producer-ns1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + Consumer blueConsumerNs2_1 = client1.newConsumer().topic(topicName2).subscriptionType(subType) + .subscriptionName("s1").subscribe(); + + // blue cluster replication consumer namespace1 + Consumer blueConsumerReplicationNs1 = client3.newConsumer().topic(topicName).subscriptionType(subType) + .subscriptionName("s1").subscribe(); + + // blue cluster replication consumer namespace2 + Consumer blueConsumerReplicationNs2 = client3.newConsumer().topic(topicName2).subscriptionType(subType) + .subscriptionName("s1").subscribe(); + + + AbstractTopic blueTopicNs1 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName, false).getNow(null).get(); + retryStrategically((test) -> !blueTopicNs1.getProducers().isEmpty(), 5, 500); + retryStrategically((test) -> !blueTopicNs1.getSubscriptions().isEmpty(), 5, 500); + assertFalse(blueTopicNs1.getProducers().isEmpty()); + assertFalse(blueTopicNs1.getSubscriptions().isEmpty()); + + AbstractTopic blueTopicNs2 = (AbstractTopic) pulsar1.getBrokerService().getTopic(topicName2, false).getNow(null).get(); + retryStrategically((test) -> !blueTopicNs2.getProducers().isEmpty(), 5, 500); + retryStrategically((test) -> !blueTopicNs2.getSubscriptions().isEmpty(), 5, 500); + assertFalse(blueTopicNs2.getProducers().isEmpty()); + assertFalse(blueTopicNs2.getSubscriptions().isEmpty()); + + // build backlog + blueConsumerNs1_1.close(); + blueConsumerNs2_1.close(); + retryStrategically((test) -> blueTopicNs1.getReplicators().size() == 1, 10, 3000); + assertEquals(blueTopicNs1.getReplicators().size(), 1); + retryStrategically((test) -> blueTopicNs2.getReplicators().size() == 1, 10, 3000); + assertEquals(blueTopicNs2.getReplicators().size(), 1); + + // stop service in the replication cluster to build replication backlog + broker3.cleanup(); + retryStrategically((test) -> broker3.getPulsarService() == null, 10, 1000); + assertNull(pulsar3.getBrokerService()); + + //publish messages into topic in blue cluster + int n = 5; + for (int i = 0; i < n; i++) { + blueProducerNs1_1.send("test1".getBytes()); + blueProducerNs2_1.send("test1".getBytes()); + } + retryStrategically((test) -> blueTopicNs1.isReplicationBacklogExist(), 10, 1000); + assertTrue(blueTopicNs1.isReplicationBacklogExist()); + retryStrategically((test) -> blueTopicNs2.isReplicationBacklogExist(), 10, 1000); + assertTrue(blueTopicNs2.isReplicationBacklogExist()); + + @Cleanup + PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + // green cluster - namespace1 - producer/consumer + Producer greenProducerNs1_1 = client2.newProducer().topic(topicName).enableBatching(false) + .producerName("green-producer-ns1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + AbstractTopic greenTopicNs1 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName, false).getNow(null).get(); + Producer greenProducerNs2_1 = client2.newProducer().topic(topicName2).enableBatching(false) + .producerName("green-producer-ns2-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + AbstractTopic greenTopicNs2 = (AbstractTopic) pulsar2.getBrokerService().getTopic(topicName2, false).getNow(null).get(); + log.info("name of topic 2 - {}", greenTopicNs1.getName()); + assertFalse(greenTopicNs1.getProducers().isEmpty()); + + retryStrategically((test) -> greenTopicNs1.getReplicators().size() == 1, 10, 2000); + log.info("replicators should be ready"); + + ClusterUrl migratedUrl = new ClusterUrl(pulsar2.getWebServiceAddress(), pulsar2.getWebServiceAddressTls(), + pulsar2.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrlTls()); + admin1.clusters().updateClusterMigration("r1", isClusterMigrate, migratedUrl); + admin1.namespaces().updateMigrationState(namespace, isNamespaceMigrate); + log.info("update cluster migration called"); + + retryStrategically((test) -> { + try { + blueTopicNs1.checkClusterMigration().get(); + if (isClusterMigrate) { + blueTopicNs2.checkClusterMigration().get(); + } + return true; + } catch (Exception e) { + // ok + } + return false; + }, 10, 500); + + blueTopicNs1.checkClusterMigration().get(); + if (isClusterMigrate) { + blueTopicNs2.checkClusterMigration().get(); + } + + blueProducerNs1_1.sendAsync("test1".getBytes()); + blueProducerNs2_1.sendAsync("test1".getBytes()); + + // producer is disconnected from blue + retryStrategically((test) -> blueTopicNs1.getProducers().isEmpty(), 10, 500); + assertTrue(blueTopicNs1.getProducers().isEmpty()); + if (isClusterMigrate) { + retryStrategically((test) -> blueTopicNs2.getProducers().isEmpty(), 10, 500); + assertTrue(blueTopicNs2.getProducers().isEmpty()); + } else { + retryStrategically((test) -> !blueTopicNs2.getProducers().isEmpty(), 10, 500); + assertFalse(blueTopicNs2.getProducers().isEmpty()); + } + + // verify that the disconnected producer is not redirected + // to replication cluster since there is replication backlog. + assertEquals(greenTopicNs1.getProducers().size(), 1); + + // Restart the service in cluster "r3". + broker3.restart(); + retryStrategically((test) -> broker3.getPulsarService() != null, 10, 1000); + assertNotNull(broker3.getPulsarService()); + pulsar3 = broker3.getPulsarService(); + + // verify that the replication backlog drains once service in cluster "r3" is restarted. + retryStrategically((test) -> !blueTopicNs1.isReplicationBacklogExist(), 10, 1000); + assertFalse(blueTopicNs1.isReplicationBacklogExist()); + retryStrategically((test) -> !blueTopicNs2.isReplicationBacklogExist(), 10, 1000); + assertFalse(blueTopicNs2.isReplicationBacklogExist()); + + // verify that the producer1 is now is now connected to migrated cluster green since backlog is cleared. + retryStrategically((test) -> greenTopicNs1.getProducers().size()==2, 10, 500); + assertEquals(greenTopicNs1.getProducers().size(), 2); + if (isClusterMigrate) { + retryStrategically((test) -> greenTopicNs2.getProducers().size()==2, 10, 500); + assertEquals(greenTopicNs2.getProducers().size(), 2); + } else { + retryStrategically((test) -> greenTopicNs2.getProducers().size()==1, 10, 500); + assertEquals(greenTopicNs2.getProducers().size(), 1); + } + + // clean up + blueProducerNs1_1.close(); + blueProducerNs2_1.close(); + blueConsumerNs1_1.close(); + blueConsumerNs2_1.close(); + greenProducerNs1_1.close(); + greenProducerNs2_1.close(); + } + static class TestBroker extends MockedPulsarServiceBaseTest { private String clusterName; 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 2690df658b7be..7f31f3e8d2d57 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 @@ -1234,7 +1234,7 @@ CompletableFuture> getAntiAffinityNamespacesAsync( * @param namespace * Namespace name * @param enableDeduplication - * wether to enable or disable deduplication feature + * whether to enable or disable deduplication feature */ CompletableFuture setDeduplicationStatusAsync(String namespace, boolean enableDeduplication); @@ -4623,4 +4623,29 @@ void setIsAllowAutoUpdateSchema(String namespace, boolean isAllowAutoUpdateSchem * @return */ CompletableFuture removeNamespaceEntryFiltersAsync(String namespace); + + /** + * Enable migration for all topics within a namespace. + *

+ * Migrate all topics of a namespace to new broker. + *

+ * Request example: + * + *

+     * true
+     * 
+ * + * @param namespace + * Namespace name + * @param migrated + * Flag to determine namespace is migrated or not + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Namespace does not exist + * @throws PulsarAdminException + * Unexpected error + */ + void updateMigrationState(String namespace, boolean migrated) throws PulsarAdminException; + } 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 066fdf1df4f09..138e8c4793010 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 @@ -126,6 +126,8 @@ public class Policies { @SuppressWarnings("checkstyle:MemberName") public String resource_group_name = null; + public boolean isMigrated; + public enum BundleType { LARGEST, HOT; } 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 59f0ef3b34763..05e4352e5fabe 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 @@ -1900,6 +1900,17 @@ public CompletableFuture removeNamespaceResourceGroupAsync(String namespac return asyncDeleteRequest(path); } + @Override + public void updateMigrationState(String namespace, boolean migrated) throws PulsarAdminException { + sync(() -> updateMigrationStateAsync(namespace, migrated)); + } + + public CompletableFuture updateMigrationStateAsync(String namespace, boolean migrated) { + NamespaceName ns = NamespaceName.get(namespace); + WebTarget path = namespacePath(ns, "migration"); + return asyncPostRequest(path, Entity.entity(migrated, MediaType.APPLICATION_JSON)); + } + private WebTarget namespacePath(NamespaceName namespace, String... parts) { final WebTarget base = namespace.isV2() ? adminV2Namespaces : adminNamespaces; WebTarget namespacePath = base.path(namespace.toString()); 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 c578876b382fb..033146aa607b0 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 @@ -159,21 +159,20 @@ private class UpdateClusterMigration extends CliCommand { @Parameter(description = "cluster-name", required = true) private java.util.List params; - @Parameter(names = "--migrated", description = "Is cluster migrated", required = true) + @Parameter(names = "--migrated", description = "Is cluster migrated") private boolean migrated; - @Parameter(names = "--service-url", description = "New migrated cluster service url", required = false) + @Parameter(names = "--service-url", description = "New migrated cluster service url") private String serviceUrl; @Parameter(names = "--service-url-secure", - description = "New migrated cluster service url secure", required = false) + description = "New migrated cluster service url secure") private String serviceUrlTls; - @Parameter(names = "--broker-url", description = "New migrated cluster broker service url", required = false) + @Parameter(names = "--broker-url", description = "New migrated cluster broker service url") private String brokerServiceUrl; - @Parameter(names = "--broker-url-secure", description = "New migrated cluster broker service url secure", - required = false) + @Parameter(names = "--broker-url-secure", description = "New migrated cluster broker service url secure") private String brokerServiceUrlTls; void run() throws PulsarAdminException { 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 33277fdb60839..8162b4b19c21d 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 @@ -2619,6 +2619,20 @@ void run() throws PulsarAdminException { getAdmin().namespaces().removeNamespaceResourceGroup(namespace); } } + @Parameters(commandDescription = "Update migration state for a namespace") + private class UpdateMigrationState extends CliCommand { + @Parameter(description = "tenant/namespace", required = true) + private java.util.List params; + + @Parameter(names = "--migrated", description = "Is namespace migrated") + private boolean migrated; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + getAdmin().namespaces().updateMigrationState(namespace, migrated); + } + } @Parameters(commandDescription = "Get entry filters for a namespace") private class GetEntryFiltersPerTopic extends CliCommand { @@ -2844,5 +2858,7 @@ public CmdNamespaces(Supplier admin) { jcommander.addCommand("get-entry-filters", new GetEntryFiltersPerTopic()); jcommander.addCommand("set-entry-filters", new SetEntryFiltersPerTopic()); jcommander.addCommand("remove-entry-filters", new RemoveEntryFiltersPerTopic()); + + jcommander.addCommand("update-migration-state", new UpdateMigrationState()); } } From a7acdf6dd60fb46fba681445d02cdbb131b9f1be Mon Sep 17 00:00:00 2001 From: "Kim, Joo Hyuk" Date: Mon, 16 Oct 2023 20:53:41 +0900 Subject: [PATCH 14/37] [refactor][cli][PIP-280] Refactor `pulsar-client-tools` module (#20764) Signed-off-by: tison Co-authored-by: tison --- pulsar-client-tools/pom.xml | 5 + .../apache/pulsar/admin/cli/CliCommand.java | 42 --- .../pulsar/admin/cli/CmdNamespaces.java | 234 +++++---------- .../pulsar/admin/cli/CmdPersistentTopics.java | 17 +- .../pulsar/admin/cli/CmdTopicPolicies.java | 127 ++++---- .../apache/pulsar/admin/cli/CmdTopics.java | 271 +++++------------- 6 files changed, 206 insertions(+), 490 deletions(-) diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 4a35523131ad2..639dba3cab7e6 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -53,6 +53,11 @@ pulsar-client-admin-original ${project.version}
+ + ${project.groupId} + pulsar-cli-utils + ${project.version} + commons-io commons-io diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java index 3d87ce644f226..e984f114c27b1 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java @@ -22,7 +22,6 @@ import com.beust.jcommander.ParameterException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; -import com.google.common.collect.Sets; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -78,46 +77,6 @@ static String validateNonPersistentTopic(List params) { return topicName.toString(); } - static void validateLatencySampleRate(int sampleRate) { - if (sampleRate < 0) { - throw new ParameterException( - "Latency sample rate should be positive and non-zero (found " + sampleRate + ")"); - } - } - - static long validateSizeString(String s) { - char last = s.charAt(s.length() - 1); - String subStr = s.substring(0, s.length() - 1); - long size; - try { - size = SIZE_UNIT.contains(last) ? Long.parseLong(subStr) : Long.parseLong(s); - } catch (IllegalArgumentException e) { - throw new ParameterException( - String.format("Invalid size '%s'. Valid formats are: %s", - s, "(4096, 100K, 10M, 16G, 2T)")); - } - switch (last) { - case 'k': - case 'K': - return size * 1024; - - case 'm': - case 'M': - return size * 1024 * 1024; - - case 'g': - case 'G': - return size * 1024 * 1024 * 1024; - - case 't': - case 'T': - return size * 1024 * 1024 * 1024 * 1024; - - default: - return size; - } - } - static MessageId validateMessageIdString(String resetMessageIdStr) throws PulsarAdminException { return validateMessageIdString(resetMessageIdStr, -1); } @@ -230,7 +189,6 @@ void prettyPrint(T item) { private static final ObjectMapper MAPPER = ObjectMapperFactory.create(); private static final ObjectWriter WRITER = MAPPER.writerWithDefaultPrettyPrinter(); - private static final Set SIZE_UNIT = Sets.newHashSet('k', 'K', 'm', 'M', 'g', 'G', 't', 'T'); abstract void run() throws Exception; } 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 8162b4b19c21d..c7658cbf77055 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 @@ -18,8 +18,6 @@ */ package org.apache.pulsar.admin.cli; -import static org.apache.pulsar.admin.cli.utils.CmdUtils.maxValueCheck; -import static org.apache.pulsar.admin.cli.utils.CmdUtils.positiveCheck; import com.beust.jcommander.Parameter; import com.beust.jcommander.ParameterException; import com.beust.jcommander.Parameters; @@ -38,6 +36,15 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.admin.cli.utils.IOUtils; +import org.apache.pulsar.cli.converters.ByteUnitIntegerConverter; +import org.apache.pulsar.cli.converters.ByteUnitToLongConverter; +import org.apache.pulsar.cli.converters.TimeUnitToMillisConverter; +import org.apache.pulsar.cli.converters.TimeUnitToSecondsConverter; +import org.apache.pulsar.cli.validators.IntegerMaxValueLongValidator; +import org.apache.pulsar.cli.validators.MinNegativeOneValidator; +import org.apache.pulsar.cli.validators.NonNegativeValueValidator; +import org.apache.pulsar.cli.validators.PositiveIntegerValueValidator; +import org.apache.pulsar.cli.validators.PositiveLongValueValidator; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.Mode; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -67,7 +74,6 @@ import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; import org.apache.pulsar.common.policies.data.TopicType; -import org.apache.pulsar.common.util.RelativeTimeUtil; @Parameters(commandDescription = "Operations about namespaces") public class CmdNamespaces extends CmdBase { @@ -404,25 +410,15 @@ private class SetMessageTTL extends CliCommand { @Parameter(names = { "--messageTTL", "-ttl" }, description = "Message TTL in seconds (or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w). " - + "When the value is set to `0`, TTL is disabled.", required = true) - private String messageTTLStr; + + "When the value is set to `0`, TTL is disabled.", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = {NonNegativeValueValidator.class}) + private Long messageTTLInSecond; @Override void run() throws PulsarAdminException { - long messageTTLInSecond; - try { - messageTTLInSecond = RelativeTimeUtil.parseRelativeTimeInSeconds(messageTTLStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - - if (messageTTLInSecond < 0 || messageTTLInSecond > Integer.MAX_VALUE) { - throw new ParameterException( - String.format("Message TTL cannot be negative or greater than %d seconds", Integer.MAX_VALUE)); - } - String namespace = validateNamespace(params); - getAdmin().namespaces().setNamespaceMessageTTL(namespace, (int) messageTTLInSecond); + getAdmin().namespaces().setNamespaceMessageTTL(namespace, messageTTLInSecond.intValue()); } } @@ -747,39 +743,27 @@ private class SetRetention extends CliCommand { + "For example, 100m, 3h, 2d, 5w. " + "If the time unit is not specified, the default unit is seconds. For example, " + "-t 120 sets retention to 2 minutes. " - + "0 means no retention and -1 means infinite time retention.", required = true) - private String retentionTimeStr; + + "0 means no retention and -1 means infinite time retention.", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = MinNegativeOneValidator.class) + private Long retentionTimeInSec; @Parameter(names = { "--size", "-s" }, description = "Retention size limit with optional size unit suffix. " + "For example, 4096, 10M, 16G, 3T. The size unit suffix character can be k/K, m/M, g/G, or t/T. " + "If the size unit suffix is not specified, the default unit is bytes. " - + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true) - private String limitStr; + + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true, + converter = ByteUnitIntegerConverter.class) + private Integer sizeLimit; @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - long sizeLimit = validateSizeString(limitStr); - long retentionTimeInSec; - try { - retentionTimeInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(retentionTimeStr); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - - final int retentionTimeInMin; - if (retentionTimeInSec != -1) { - retentionTimeInMin = (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec); - } else { - retentionTimeInMin = -1; - } - - final int retentionSizeInMB; - if (sizeLimit != -1) { - retentionSizeInMB = (int) (sizeLimit / (1024 * 1024)); - } else { - retentionSizeInMB = -1; - } + final int retentionTimeInMin = retentionTimeInSec != -1 + ? (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec) + : retentionTimeInSec.intValue(); + final int retentionSizeInMB = sizeLimit != -1 + ? (int) (sizeLimit / (1024 * 1024)) + : sizeLimit; getAdmin().namespaces() .setRetention(namespace, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)); } @@ -1267,13 +1251,15 @@ private class SetBacklogQuota extends CliCommand { @Parameter(description = "tenant/namespace", required = true) private java.util.List params; - @Parameter(names = { "-l", "--limit" }, description = "Size limit (eg: 10M, 16G)") - private String limitStr; + @Parameter(names = { "-l", "--limit" }, description = "Size limit (eg: 10M, 16G)", + converter = ByteUnitToLongConverter.class) + private Long limit = 0L; @Parameter(names = { "-lt", "--limitTime" }, description = "Time limit in second (or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w), " - + "non-positive number for disabling time limit.") - private String limitTimeStr = null; + + "non-positive number for disabling time limit.", + converter = TimeUnitToSecondsConverter.class) + private Long limitTimeInSec; @Parameter(names = { "-p", "--policy" }, description = "Retention policy to enforce when the limit is reached. " + "Valid options are: [producer_request_hold, producer_exception, consumer_backlog_eviction]", @@ -1311,23 +1297,16 @@ void run() throws PulsarAdminException { BacklogQuota.Builder builder = BacklogQuota.builder().retentionPolicy(policy); if (backlogQuotaType == BacklogQuota.BacklogQuotaType.destination_storage) { // set quota by storage size - if (limitStr == null) { + if (limit == null) { throw new ParameterException("Quota type of 'destination_storage' needs a size limit"); } - long limit = validateSizeString(limitStr); builder.limitSize(limit); } else { // set quota by time - if (limitTimeStr == null) { + if (limitTimeInSec == null) { throw new ParameterException("Quota type of 'message_age' needs a time limit"); } - long limitTimeInSec; - try { - limitTimeInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(limitTimeStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - builder.limitTime((int) limitTimeInSec); + builder.limitTime(limitTimeInSec.intValue()); } getAdmin().namespaces().setBacklogQuota(namespace, builder.build(), backlogQuotaType); } @@ -1572,8 +1551,10 @@ private class SetInactiveTopicPolicies extends CliCommand { @Parameter(names = {"--max-inactive-duration", "-t"}, description = "Max duration of topic inactivity in " + "seconds, topics that are inactive for longer than this value will be deleted " - + "(eg: 1s, 10s, 1m, 5h, 3d)", required = true) - private String deleteInactiveTopicsMaxInactiveDuration; + + "(eg: 1s, 10s, 1m, 5h, 3d)", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = IntegerMaxValueLongValidator.class) + private Long maxInactiveDurationInSeconds; @Parameter(names = { "--delete-mode", "-m" }, description = "Mode of delete inactive topic, Valid options are: " + "[delete_when_no_subscriptions, delete_when_subscriptions_caught_up]", required = true) @@ -1582,14 +1563,6 @@ private class SetInactiveTopicPolicies extends CliCommand { @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - long maxInactiveDurationInSeconds; - try { - maxInactiveDurationInSeconds = TimeUnit.SECONDS.toSeconds( - RelativeTimeUtil.parseRelativeTimeInSeconds(deleteInactiveTopicsMaxInactiveDuration)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (enableDeleteWhileInactive == disableDeleteWhileInactive) { throw new ParameterException("Need to specify either enable-delete-while-inactive or " + "disable-delete-while-inactive"); @@ -1602,7 +1575,7 @@ void run() throws PulsarAdminException { + "delete_when_subscriptions_caught_up"); } getAdmin().namespaces().setInactiveTopicPolicies(namespace, new InactiveTopicPolicies(deleteMode, - (int) maxInactiveDurationInSeconds, enableDeleteWhileInactive)); + maxInactiveDurationInSeconds.intValue(), enableDeleteWhileInactive)); } } @@ -1619,20 +1592,13 @@ private class SetDelayedDelivery extends CliCommand { @Parameter(names = { "--time", "-t" }, description = "The tick time for when retrying on " + "delayed delivery messages, affecting the accuracy of the delivery time compared to " - + "the scheduled time. (eg: 1s, 10s, 1m, 5h, 3d)") - private String delayedDeliveryTimeStr = "1s"; + + "the scheduled time. (eg: 1s, 10s, 1m, 5h, 3d)", + converter = TimeUnitToMillisConverter.class) + private Long delayedDeliveryTimeInMills = 1000L; @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - long delayedDeliveryTimeInMills; - try { - delayedDeliveryTimeInMills = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(delayedDeliveryTimeStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (enable == disable) { throw new ParameterException("Need to specify either --enable or --disable"); } @@ -1944,13 +1910,13 @@ private class SetCompactionThreshold extends CliCommand { @Parameter(names = { "--threshold", "-t" }, description = "Maximum number of bytes in a topic backlog before compaction is triggered " + "(eg: 10M, 16G, 3T). 0 disables automatic compaction", - required = true) - private String thresholdStr = "0"; + required = true, + converter = ByteUnitToLongConverter.class) + private Long threshold = 0L; @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - long threshold = validateSizeString(thresholdStr); getAdmin().namespaces().setCompactionThreshold(namespace, threshold); } } @@ -1978,13 +1944,13 @@ private class SetOffloadThreshold extends CliCommand { + " -1 falls back to the cluster's namespace default." + " Negative values disable automatic offload." + " 0 triggers offloading as soon as possible.", - required = true) - private String thresholdStr = "-1"; + required = true, + converter = ByteUnitToLongConverter.class) + private Long threshold = -1L; @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - long threshold = validateSizeString(thresholdStr); getAdmin().namespaces().setOffloadThreshold(namespace, threshold); } } @@ -2014,18 +1980,13 @@ private class SetOffloadDeletionLag extends CliCommand { @Parameter(names = { "--lag", "-l" }, description = "Duration to wait after offloading a ledger segment, before deleting the copy of that" + " segment from cluster local storage. (eg: 10m, 5h, 3d, 2w).", - required = true) - private String lag = "-1"; + required = true, + converter = TimeUnitToSecondsConverter.class) + private Long lagInSec = -1L; @Override void run() throws PulsarAdminException { String namespace = validateNamespace(params); - long lagInSec; - try { - lagInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(lag); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } getAdmin().namespaces().setOffloadDeleteLag(namespace, lagInSec, TimeUnit.SECONDS); } @@ -2268,34 +2229,41 @@ private class SetOffloadPolicies extends CliCommand { names = {"--maxBlockSize", "-mbs"}, description = "Max block size (eg: 32M, 64M), default is 64MB" + "s3 and google-cloud-storage requires this parameter", - required = false) - private String maxBlockSizeStr; + required = false, + converter = ByteUnitIntegerConverter.class, + validateValueWith = {PositiveIntegerValueValidator.class}) + private Integer maxBlockSizeInBytes = OffloadPoliciesImpl.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; @Parameter( names = {"--readBufferSize", "-rbs"}, description = "Read buffer size (eg: 1M, 5M), default is 1MB", - required = false) - private String readBufferSizeStr; + required = false, + converter = ByteUnitIntegerConverter.class, + validateValueWith = {PositiveIntegerValueValidator.class}) + private Integer readBufferSizeInBytes = OffloadPoliciesImpl.DEFAULT_READ_BUFFER_SIZE_IN_BYTES; @Parameter( names = {"--offloadAfterElapsed", "-oae"}, description = "Delay time in Millis for deleting the bookkeeper ledger after offload " + "(or seconds,minutes,hours,days,weeks eg: 10s, 100m, 3h, 2d, 5w).", - required = false) - private String offloadAfterElapsedStr; + required = false, + converter = TimeUnitToMillisConverter.class, + validateValueWith = PositiveLongValueValidator.class) + private Long offloadAfterElapsedInMillis = OffloadPoliciesImpl.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS; @Parameter( names = {"--offloadAfterThreshold", "-oat"}, description = "Offload after threshold size (eg: 1M, 5M)", - required = false) - private String offloadAfterThresholdStr; + required = false, + converter = ByteUnitToLongConverter.class) + private Long offloadAfterThresholdInBytes = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES; @Parameter( names = {"--offloadAfterThresholdInSeconds", "-oats"}, description = "Offload after threshold seconds (or minutes,hours,days,weeks eg: 100m, 3h, 2d, 5w).", - required = false - ) - private String offloadAfterThresholdInSecondsStr; + required = false, + converter = TimeUnitToSecondsConverter.class) + private Long offloadThresholdInSeconds = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_SECONDS; @Parameter( names = {"--offloadedReadPriority", "-orp"}, @@ -2336,71 +2304,15 @@ void run() throws PulsarAdminException { + " if s3 offload enabled"); } - int maxBlockSizeInBytes = OffloadPoliciesImpl.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; - if (StringUtils.isNotEmpty(maxBlockSizeStr)) { - long maxBlockSize = validateSizeString(maxBlockSizeStr); - if (positiveCheck("MaxBlockSize", maxBlockSize) - && maxValueCheck("MaxBlockSize", maxBlockSize, Integer.MAX_VALUE)) { - maxBlockSizeInBytes = Long.valueOf(maxBlockSize).intValue(); - } - } - - int readBufferSizeInBytes = OffloadPoliciesImpl.DEFAULT_READ_BUFFER_SIZE_IN_BYTES; - if (StringUtils.isNotEmpty(readBufferSizeStr)) { - long readBufferSize = validateSizeString(readBufferSizeStr); - if (positiveCheck("ReadBufferSize", readBufferSize) - && maxValueCheck("ReadBufferSize", readBufferSize, Integer.MAX_VALUE)) { - readBufferSizeInBytes = Long.valueOf(readBufferSize).intValue(); - } - } - - Long offloadAfterElapsedInMillis = OffloadPoliciesImpl.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS; - if (StringUtils.isNotEmpty(offloadAfterElapsedStr)) { - Long offloadAfterElapsed; - try { - offloadAfterElapsed = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(offloadAfterElapsedStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (positiveCheck("OffloadAfterElapsed", offloadAfterElapsed) - && maxValueCheck("OffloadAfterElapsed", offloadAfterElapsed, Long.MAX_VALUE)) { - offloadAfterElapsedInMillis = offloadAfterElapsed; - } - } - - Long offloadAfterThresholdInBytes = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES; - if (StringUtils.isNotEmpty(offloadAfterThresholdStr)) { - long offloadAfterThreshold = validateSizeString(offloadAfterThresholdStr); - if (maxValueCheck("OffloadAfterThreshold", offloadAfterThreshold, Long.MAX_VALUE)) { - offloadAfterThresholdInBytes = offloadAfterThreshold; - } - } - - Long offloadThresholdInSeconds = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_SECONDS; - if (StringUtils.isNotEmpty(offloadAfterThresholdInSecondsStr)) { - Long offloadThresholdInSeconds0; - try { - offloadThresholdInSeconds0 = TimeUnit.SECONDS.toSeconds( - RelativeTimeUtil.parseRelativeTimeInSeconds(offloadAfterThresholdInSecondsStr.trim())); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (maxValueCheck("OffloadAfterThresholdInSeconds", offloadThresholdInSeconds0, Long.MAX_VALUE)) { - offloadThresholdInSeconds = offloadThresholdInSeconds0; - } - } - OffloadedReadPriority offloadedReadPriority = OffloadPoliciesImpl.DEFAULT_OFFLOADED_READ_PRIORITY; - if (this.offloadReadPriorityStr != null) { try { offloadedReadPriority = OffloadedReadPriority.fromString(this.offloadReadPriorityStr); } catch (Exception e) { throw new ParameterException("--offloadedReadPriority parameter must be one of " + Arrays.stream(OffloadedReadPriority.values()) - .map(OffloadedReadPriority::toString) - .collect(Collectors.joining(",")) + .map(OffloadedReadPriority::toString) + .collect(Collectors.joining(",")) + " but got: " + this.offloadReadPriorityStr, e); } } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java index 3c1662d00a034..c9c55ff5c0d07 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdPersistentTopics.java @@ -29,9 +29,10 @@ import io.netty.buffer.Unpooled; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import org.apache.pulsar.cli.converters.TimeUnitToMillisConverter; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -541,8 +542,9 @@ private class ResetCursor extends CliCommand { @Parameter(names = { "--time", "-t" }, description = "time in minutes to reset back to " - + "(or minutes, hours,days,weeks eg: 100m, 3h, 2d, 5w)", required = false) - private String resetTimeStr; + + "(or minutes, hours,days,weeks eg: 100m, 3h, 2d, 5w)", required = false, + converter = TimeUnitToMillisConverter.class) + private Long resetTimeInMillis = null; @Parameter(names = { "--messageId", "-m" }, description = "messageId to reset back to (ledgerId:entryId)", required = false) @@ -554,14 +556,7 @@ void run() throws PulsarAdminException { if (isNotBlank(resetMessageIdStr)) { MessageId messageId = validateMessageIdString(resetMessageIdStr); getPersistentTopics().resetCursor(persistentTopic, subName, messageId); - } else if (isNotBlank(resetTimeStr)) { - long resetTimeInMillis; - try { - resetTimeInMillis = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } + } else if (Objects.nonNull(resetTimeInMillis)) { // now - go back time long timestamp = System.currentTimeMillis() - resetTimeInMillis; getPersistentTopics().resetCursor(persistentTopic, subName, timestamp); 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 2914a5e8a08b8..6120f412edf36 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 @@ -32,6 +32,13 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.cli.converters.ByteUnitIntegerConverter; +import org.apache.pulsar.cli.converters.ByteUnitToLongConverter; +import org.apache.pulsar.cli.converters.TimeUnitToMillisConverter; +import org.apache.pulsar.cli.converters.TimeUnitToSecondsConverter; +import org.apache.pulsar.cli.validators.IntegerMaxValueLongValidator; +import org.apache.pulsar.cli.validators.MinNegativeOneValidator; +import org.apache.pulsar.cli.validators.NonNegativeValueValidator; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.TopicPolicies; @@ -50,7 +57,6 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscribeRate; -import org.apache.pulsar.common.util.RelativeTimeUtil; @Parameters(commandDescription = "Operations on persistent topics") public class CmdTopicPolicies extends CmdBase { @@ -349,8 +355,10 @@ private class SetMessageTTL extends CliCommand { @Parameter(names = { "-t", "--ttl" }, description = "Message TTL for topic in seconds (or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w), " - + "allowed range from 1 to Integer.MAX_VALUE", required = true) - private String messageTTLStr; + + "allowed range from 1 to Integer.MAX_VALUE", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = {NonNegativeValueValidator.class}) + private Long messageTTLInSecond; @Parameter(names = { "--global", "-g" }, description = "Whether to set this policy globally. " + "If set to true, broker returned global topic policies") @@ -358,20 +366,8 @@ private class SetMessageTTL extends CliCommand { @Override void run() throws PulsarAdminException { - long messageTTLInSecond; - try { - messageTTLInSecond = RelativeTimeUtil.parseRelativeTimeInSeconds(messageTTLStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - - if (messageTTLInSecond < 0 || messageTTLInSecond > Integer.MAX_VALUE) { - throw new ParameterException( - String.format("Message TTL cannot be negative or greater than %d seconds", Integer.MAX_VALUE)); - } - String persistentTopic = validatePersistentTopic(params); - getTopicPolicies(isGlobal).setMessageTTL(persistentTopic, (int) messageTTLInSecond); + getTopicPolicies(isGlobal).setMessageTTL(persistentTopic, messageTTLInSecond.intValue()); } } @@ -539,14 +535,17 @@ private class SetRetention extends CliCommand { + "For example, 100m, 3h, 2d, 5w. " + "If the time unit is not specified, the default unit is seconds. For example, " + "-t 120 sets retention to 2 minutes. " - + "0 means no retention and -1 means infinite time retention.", required = true) - private String retentionTimeStr; + + "0 means no retention and -1 means infinite time retention.", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = MinNegativeOneValidator.class) + private Long retentionTimeInSec; @Parameter(names = { "--size", "-s" }, description = "Retention size limit with optional size unit suffix. " + "For example, 4096, 10M, 16G, 3T. The size unit suffix character can be k/K, m/M, g/G, or t/T. " + "If the size unit suffix is not specified, the default unit is bytes. " - + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true) - private String limitStr; + + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true, + converter = ByteUnitIntegerConverter.class) + private Integer sizeLimit; @Parameter(names = { "--global", "-g" }, description = "Whether to set this policy globally. " + "If set to true, the policy is replicated to other clusters asynchronously, " @@ -556,22 +555,12 @@ private class SetRetention extends CliCommand { @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long sizeLimit = validateSizeString(limitStr); - long retentionTimeInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(retentionTimeStr); - - final int retentionTimeInMin; - if (retentionTimeInSec != -1) { - retentionTimeInMin = (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec); - } else { - retentionTimeInMin = -1; - } - - final int retentionSizeInMB; - if (sizeLimit != -1) { - retentionSizeInMB = (int) (sizeLimit / (1024 * 1024)); - } else { - retentionSizeInMB = -1; - } + final int retentionTimeInMin = retentionTimeInSec != -1 + ? (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec) + : retentionTimeInSec.intValue(); + final int retentionSizeInMB = sizeLimit != -1 + ? (int) (sizeLimit / (1024 * 1024)) + : sizeLimit; getTopicPolicies(isGlobal).setRetention(persistentTopic, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)); } @@ -719,8 +708,9 @@ private class SetDelayedDelivery extends CliCommand { @Parameter(names = { "--time", "-t" }, description = "The tick time for when retrying on " + "delayed delivery messages, affecting the accuracy of the delivery time compared to " - + "the scheduled time. (eg: 1s, 10s, 1m, 5h, 3d)") - private String delayedDeliveryTimeStr = "1s"; + + "the scheduled time. (eg: 1s, 10s, 1m, 5h, 3d)", + converter = TimeUnitToMillisConverter.class) + private Long delayedDeliveryTimeInMills = 1000L; @Parameter(names = { "--global", "-g" }, description = "Whether to set this policy globally. " + "If set to true, the policy will be replicate to other clusters asynchronously") @@ -729,14 +719,6 @@ private class SetDelayedDelivery extends CliCommand { @Override void run() throws PulsarAdminException { String topicName = validateTopicName(params); - long delayedDeliveryTimeInMills; - try { - delayedDeliveryTimeInMills = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(delayedDeliveryTimeStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (enable == disable) { throw new ParameterException("Need to specify either --enable or --disable"); } @@ -966,13 +948,16 @@ private class SetBacklogQuota extends CliCommand { @Parameter(description = "persistent://tenant/namespace/topic", required = true) private java.util.List params; - @Parameter(names = { "-l", "--limit" }, description = "Size limit (eg: 10M, 16G)") - private String limitStr = null; + @Parameter(names = { "-l", "--limit" }, description = "Size limit (eg: 10M, 16G)", + converter = ByteUnitToLongConverter.class) + private Long limit; @Parameter(names = { "-lt", "--limitTime" }, description = "Time limit in second (or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w), " - + "non-positive number for disabling time limit.") - private String limitTimeStr = null; + + "non-positive number for disabling time limit.", + converter = TimeUnitToSecondsConverter.class, + validateValueWith = IntegerMaxValueLongValidator.class) + private Long limitTimeInSec; @Parameter(names = { "-p", "--policy" }, description = "Retention policy to enforce when the limit is reached. " + "Valid options are: [producer_request_hold, producer_exception, consumer_backlog_eviction]", @@ -1012,27 +997,16 @@ void run() throws PulsarAdminException { if (backlogQuotaType == BacklogQuota.BacklogQuotaType.destination_storage) { // set quota by storage size - if (limitStr == null) { + if (limit == null) { throw new ParameterException("Quota type of 'destination_storage' needs a size limit"); } - long limit = validateSizeString(limitStr); - builder.limitSize((int) limit); + builder.limitSize(limit); } else { // set quota by time - if (limitTimeStr == null) { + if (limitTimeInSec == null) { throw new ParameterException("Quota type of 'message_age' needs a time limit"); } - long limitTimeInSec; - try { - limitTimeInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(limitTimeStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - if (limitTimeInSec > Integer.MAX_VALUE) { - throw new ParameterException( - String.format("Time limit cannot be greater than %d seconds", Integer.MAX_VALUE)); - } - builder.limitTime((int) limitTimeInSec); + builder.limitTime(limitTimeInSec.intValue()); } getTopicPolicies(isGlobal).setBacklogQuota(persistentTopic, builder.build(), @@ -1274,8 +1248,10 @@ private class SetCompactionThreshold extends CliCommand { @Parameter(names = { "--threshold", "-t" }, description = "Maximum number of bytes in a topic backlog before compaction is triggered " + "(eg: 10M, 16G, 3T). 0 disables automatic compaction", - required = true) - private String thresholdStr = "0"; + required = true, + converter = ByteUnitToLongConverter.class) + private Long threshold = 0L; + @Parameter(names = { "--global", "-g" }, description = "Whether to set this policy globally. " + "If set to true, the policy will be replicate to other clusters asynchronously") private boolean isGlobal = false; @@ -1283,7 +1259,6 @@ private class SetCompactionThreshold extends CliCommand { @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long threshold = validateSizeString(thresholdStr); getTopicPolicies(isGlobal).setCompactionThreshold(persistentTopic, threshold); } } @@ -1411,8 +1386,10 @@ private class SetInactiveTopicPolicies extends CliCommand { @Parameter(names = {"--max-inactive-duration", "-t"}, description = "Max duration of topic inactivity in seconds, topics that are inactive for longer than " - + "this value will be deleted (eg: 1s, 10s, 1m, 5h, 3d)", required = true) - private String deleteInactiveTopicsMaxInactiveDuration; + + "this value will be deleted (eg: 1s, 10s, 1m, 5h, 3d)", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = IntegerMaxValueLongValidator.class) + private Long maxInactiveDurationInSeconds; @Parameter(names = { "--delete-mode", "-m" }, description = "Mode of delete inactive topic, Valid options are: " + "[delete_when_no_subscriptions, delete_when_subscriptions_caught_up]", required = true) @@ -1425,14 +1402,6 @@ private class SetInactiveTopicPolicies extends CliCommand { @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long maxInactiveDurationInSeconds; - try { - maxInactiveDurationInSeconds = TimeUnit.SECONDS.toSeconds( - RelativeTimeUtil.parseRelativeTimeInSeconds(deleteInactiveTopicsMaxInactiveDuration)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (enableDeleteWhileInactive == disableDeleteWhileInactive) { throw new ParameterException("Need to specify either enable-delete-while-inactive or " + "disable-delete-while-inactive"); @@ -1445,7 +1414,7 @@ void run() throws PulsarAdminException { + "delete_when_subscriptions_caught_up"); } getTopicPolicies(isGlobal).setInactiveTopicPolicies(persistentTopic, new InactiveTopicPolicies(deleteMode, - (int) maxInactiveDurationInSeconds, enableDeleteWhileInactive)); + maxInactiveDurationInSeconds.intValue(), enableDeleteWhileInactive)); } } 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 31d9648babcff..1aa905c647876 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 @@ -40,6 +40,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -49,6 +50,14 @@ import java.util.stream.Collectors; import lombok.Getter; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.cli.converters.ByteUnitIntegerConverter; +import org.apache.pulsar.cli.converters.ByteUnitToLongConverter; +import org.apache.pulsar.cli.converters.TimeUnitToMillisConverter; +import org.apache.pulsar.cli.converters.TimeUnitToSecondsConverter; +import org.apache.pulsar.cli.validators.IntegerMaxValueLongValidator; +import org.apache.pulsar.cli.validators.MinNegativeOneValidator; +import org.apache.pulsar.cli.validators.NonNegativeValueValidator; +import org.apache.pulsar.cli.validators.PositiveIntegerValueValidator; import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.OffloadProcessStatus; @@ -79,7 +88,6 @@ import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.RelativeTimeUtil; @Getter @Parameters(commandDescription = "Operations on persistent topics") @@ -934,8 +942,9 @@ private class ExpireMessages extends CliCommand { private String subName; @Parameter(names = { "-t", "--expireTime" }, description = "Expire messages older than time in seconds " - + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w)") - private String expireTimeStr = null; + + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w)", + converter = TimeUnitToSecondsConverter.class) + private Long expireTimeInSeconds = -1L; @Parameter(names = { "--position", "-p" }, description = "message position to reset back to (ledgerId:entryId)", required = false) @@ -947,15 +956,6 @@ private class ExpireMessages extends CliCommand { @Override void run() throws PulsarAdminException { - long expireTimeInSeconds = -1; - if (expireTimeStr != null) { - try { - expireTimeInSeconds = RelativeTimeUtil.parseRelativeTimeInSeconds(expireTimeStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - } - if (expireTimeInSeconds >= 0 && isNotBlank(messagePosition)) { throw new ParameterException(String.format("Can't expire message by time and " + "by message position at the same time.")); @@ -982,17 +982,12 @@ private class ExpireMessagesForAllSubscriptions extends CliCommand { private java.util.List params; @Parameter(names = { "-t", "--expireTime" }, description = "Expire messages older than time in seconds " - + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w)", required = true) - private String expireTimeStr; + + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w)", required = true, + converter = TimeUnitToSecondsConverter.class) + private Long expireTimeInSeconds; @Override void run() throws PulsarAdminException { - long expireTimeInSeconds; - try { - expireTimeInSeconds = RelativeTimeUtil.parseRelativeTimeInSeconds(expireTimeStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } String topic = validateTopicName(params); getTopics().expireMessagesForAllSubscriptions(topic, expireTimeInSeconds); } @@ -1099,8 +1094,9 @@ private class ResetCursor extends CliCommand { @Parameter(names = { "--time", "-t" }, description = "time in minutes to reset back to " - + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w)", required = false) - private String resetTimeStr; + + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w)", required = false, + converter = TimeUnitToMillisConverter.class) + private Long resetTimeInMillis = null; @Parameter(names = { "--messageId", "-m" }, description = "messageId to reset back to ('latest', 'earliest', or 'ledgerId:entryId')") @@ -1127,14 +1123,7 @@ void run() throws PulsarAdminException { } else { getTopics().resetCursor(persistentTopic, subName, messageId); } - } else if (isNotBlank(resetTimeStr)) { - long resetTimeInMillis; - try { - resetTimeInMillis = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } + } else if (Objects.nonNull(resetTimeInMillis)) { // now - go back time long timestamp = System.currentTimeMillis() - resetTimeInMillis; getTopics().resetCursor(persistentTopic, subName, timestamp); @@ -1457,8 +1446,9 @@ static MessageId findFirstLedgerWithinThreshold(List params; @@ -1466,7 +1456,6 @@ private class Offload extends CliCommand { @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long sizeThreshold = validateSizeString(sizeThresholdStr); PersistentTopicInternalStats stats = getTopics().getInternalStats(persistentTopic, false); if (stats.ledgers.size() < 1) { @@ -1560,13 +1549,15 @@ private class SetBacklogQuota extends CliCommand { @Parameter(description = "persistent://tenant/namespace/topic", required = true) private java.util.List params; - @Parameter(names = { "-l", "--limit" }, description = "Size limit (eg: 10M, 16G)") - private String limitStr = "-1"; + @Parameter(names = { "-l", "--limit" }, description = "Size limit (eg: 10M, 16G)", + converter = ByteUnitToLongConverter.class) + private Long limit = -1L; @Parameter(names = { "-lt", "--limitTime" }, description = "Time limit in second (or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w), " - + "non-positive number for disabling time limit.") - private String limitTimeStr = null; + + "non-positive number for disabling time limit.", + converter = TimeUnitToSecondsConverter.class, validateValueWith = IntegerMaxValueLongValidator.class) + private Long limitTimeInSec = -1L; @Parameter(names = { "-p", "--policy" }, description = "Retention policy to enforce when the limit is reached. Valid options are: " @@ -1583,7 +1574,6 @@ private class SetBacklogQuota extends CliCommand { @Override void run() throws PulsarAdminException { BacklogQuota.RetentionPolicy policy; - long limit; BacklogQuota.BacklogQuotaType backlogQuotaType; try { @@ -1593,8 +1583,6 @@ void run() throws PulsarAdminException { policyStr, Arrays.toString(BacklogQuota.RetentionPolicy.values()))); } - limit = validateSizeString(limitStr); - try { backlogQuotaType = BacklogQuota.BacklogQuotaType.valueOf(backlogQuotaTypeStr); } catch (IllegalArgumentException e) { @@ -1602,23 +1590,10 @@ void run() throws PulsarAdminException { backlogQuotaTypeStr, Arrays.toString(BacklogQuota.BacklogQuotaType.values()))); } - long limitTimeInSec = -1; - if (limitTimeStr != null) { - try { - limitTimeInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(limitTimeStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - } - if (limitTimeInSec > Integer.MAX_VALUE) { - throw new ParameterException( - String.format("Time limit cannot be greater than %d seconds", Integer.MAX_VALUE)); - } - String persistentTopic = validatePersistentTopic(params); getTopics().setBacklogQuota(persistentTopic, BacklogQuota.builder().limitSize(limit) - .limitTime((int) limitTimeInSec) + .limitTime(limitTimeInSec.intValue()) .retentionPolicy(policy) .build(), backlogQuotaType); @@ -1786,20 +1761,13 @@ private class SetDelayedDelivery extends CliCommand { @Parameter(names = { "--time", "-t" }, description = "The tick time for when retrying on delayed delivery messages, affecting the accuracy of " - + "the delivery time compared to the scheduled time. (eg: 1s, 10s, 1m, 5h, 3d)") - private String delayedDeliveryTimeStr = "1s"; + + "the delivery time compared to the scheduled time. (eg: 1s, 10s, 1m, 5h, 3d)", + converter = TimeUnitToMillisConverter.class) + private Long delayedDeliveryTimeInMills = 1_000L; @Override void run() throws PulsarAdminException { String topicName = validateTopicName(params); - long delayedDeliveryTimeInMills; - try { - delayedDeliveryTimeInMills = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(delayedDeliveryTimeStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (enable == disable) { throw new ParameterException("Need to specify either --enable or --disable"); } @@ -1845,25 +1813,15 @@ private class SetMessageTTL extends CliCommand { @Parameter(names = { "-t", "--ttl" }, description = "Message TTL for topic in second " + "(or minutes, hours, days, weeks eg: 100m, 3h, 2d, 5w), " - + "allowed range from 1 to Integer.MAX_VALUE", required = true) - private String messageTTLStr; + + "allowed range from 1 to Integer.MAX_VALUE", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = {NonNegativeValueValidator.class, IntegerMaxValueLongValidator.class}) + private Long messageTTLInSecond; @Override void run() throws PulsarAdminException { - long messageTTLInSecond; - try { - messageTTLInSecond = RelativeTimeUtil.parseRelativeTimeInSeconds(messageTTLStr); - } catch (IllegalArgumentException e) { - throw new ParameterException(e.getMessage()); - } - - if (messageTTLInSecond < 0 || messageTTLInSecond > Integer.MAX_VALUE) { - throw new ParameterException( - String.format("Message TTL cannot be negative or greater than %d seconds", Integer.MAX_VALUE)); - } - String persistentTopic = validatePersistentTopic(params); - getTopics().setMessageTTL(persistentTopic, (int) messageTTLInSecond); + getTopics().setMessageTTL(persistentTopic, messageTTLInSecond.intValue()); } } @@ -1950,39 +1908,27 @@ private class SetRetention extends CliCommand { + "For example, 100m, 3h, 2d, 5w. " + "If the time unit is not specified, the default unit is seconds. For example, " + "-t 120 will set retention to 2 minutes. " - + "0 means no retention and -1 means infinite time retention.", required = true) - private String retentionTimeStr; + + "0 means no retention and -1 means infinite time retention.", required = true, + converter = TimeUnitToSecondsConverter.class, + validateValueWith = MinNegativeOneValidator.class) + private Integer retentionTimeInSec; @Parameter(names = { "--size", "-s" }, description = "Retention size limit with optional size unit suffix. " + "For example, 4096, 10M, 16G, 3T. The size unit suffix character can be k/K, m/M, g/G, or t/T. " + "If the size unit suffix is not specified, the default unit is bytes. " - + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true) - private String limitStr; + + "0 or less than 1MB means no retention and -1 means infinite size retention", required = true, + converter = ByteUnitIntegerConverter.class) + private Integer sizeLimit; @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long sizeLimit = validateSizeString(limitStr); - long retentionTimeInSec; - try { - retentionTimeInSec = RelativeTimeUtil.parseRelativeTimeInSeconds(retentionTimeStr); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - - final int retentionTimeInMin; - if (retentionTimeInSec != -1) { - retentionTimeInMin = (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec); - } else { - retentionTimeInMin = -1; - } - - final int retentionSizeInMB; - if (sizeLimit != -1) { - retentionSizeInMB = (int) (sizeLimit / (1024 * 1024)); - } else { - retentionSizeInMB = -1; - } + final int retentionTimeInMin = retentionTimeInSec != -1 + ? (int) TimeUnit.SECONDS.toMinutes(retentionTimeInSec) + : retentionTimeInSec.intValue(); + final int retentionSizeInMB = sizeLimit != -1 + ? (int) (sizeLimit / (1024 * 1024)) + : sizeLimit; getTopics().setRetention(persistentTopic, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)); } } @@ -2150,28 +2096,35 @@ private class SetOffloadPolicies extends CliCommand { names = {"-m", "--maxBlockSizeInBytes", "--maxBlockSize", "-mbs"}, description = "Max block size (eg: 32M, 64M), default is 64MB" + "s3 and google-cloud-storage requires this parameter", - required = false) - private String maxBlockSizeStr; + required = false, + converter = ByteUnitIntegerConverter.class, + validateValueWith = PositiveIntegerValueValidator.class) + private Integer maxBlockSizeInBytes = OffloadPoliciesImpl.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; @Parameter( names = {"-rb", "--readBufferSizeInBytes", "--readBufferSize", "-rbs"}, description = "Read buffer size (eg: 1M, 5M), default is 1MB" + "s3 and google-cloud-storage requires this parameter", - required = false) - private String readBufferSizeStr; + required = false, + converter = ByteUnitIntegerConverter.class, + validateValueWith = PositiveIntegerValueValidator.class) + private Integer readBufferSizeInBytes = OffloadPoliciesImpl.DEFAULT_READ_BUFFER_SIZE_IN_BYTES; @Parameter(names = {"-t", "--offloadThresholdInBytes", "--offloadAfterThreshold", "-oat"} - , description = "Offload after threshold size (eg: 1M, 5M)", required = false) - private String offloadAfterThresholdStr; + , description = "Offload after threshold size (eg: 1M, 5M)", required = false, + converter = ByteUnitToLongConverter.class) + private Long offloadAfterThresholdInBytes = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES; @Parameter(names = {"-ts", "--offloadThresholdInSeconds", "--offloadAfterThresholdInSeconds", "-oats"}, - description = "Offload after threshold seconds (or minutes,hours,days,weeks eg: 100m, 3h, 2d, 5w).") - private String offloadAfterThresholdInSecondsStr; + description = "Offload after threshold seconds (or minutes,hours,days,weeks eg: 100m, 3h, 2d, 5w).", + converter = TimeUnitToSecondsConverter.class) + private Long offloadThresholdInSeconds = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_SECONDS; @Parameter(names = {"-dl", "--offloadDeletionLagInMillis", "--offloadAfterElapsed", "-oae"} , description = "Delay time in Millis for deleting the bookkeeper ledger after offload " - + "(or seconds,minutes,hours,days,weeks eg: 10s, 100m, 3h, 2d, 5w).") - private String offloadAfterElapsedStr; + + "(or seconds,minutes,hours,days,weeks eg: 10s, 100m, 3h, 2d, 5w).", + converter = TimeUnitToMillisConverter.class) + private Long offloadAfterElapsedInMillis = OffloadPoliciesImpl.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS; @Parameter(names = {"--offloadedReadPriority", "-orp"}, description = "Read priority for offloaded messages. " @@ -2196,20 +2149,6 @@ public boolean isS3Driver(String driver) { return driver.equalsIgnoreCase(driverNames.get(0)) || driver.equalsIgnoreCase(driverNames.get(1)); } - public boolean positiveCheck(String paramName, long value) { - if (value <= 0) { - throw new ParameterException(paramName + " cannot be less than or equal to 0!"); - } - return true; - } - - public boolean maxValueCheck(String paramName, long value, long maxValue) { - if (value > maxValue) { - throw new ParameterException(paramName + " cannot be greater than " + maxValue + "!"); - } - return true; - } - @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); @@ -2225,62 +2164,7 @@ void run() throws PulsarAdminException { + " if s3 offload enabled"); } - int maxBlockSizeInBytes = OffloadPoliciesImpl.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; - if (StringUtils.isNotEmpty(maxBlockSizeStr)) { - long maxBlockSize = validateSizeString(maxBlockSizeStr); - if (positiveCheck("MaxBlockSize", maxBlockSize) - && maxValueCheck("MaxBlockSize", maxBlockSize, Integer.MAX_VALUE)) { - maxBlockSizeInBytes = Long.valueOf(maxBlockSize).intValue(); - } - } - - int readBufferSizeInBytes = OffloadPoliciesImpl.DEFAULT_READ_BUFFER_SIZE_IN_BYTES; - if (StringUtils.isNotEmpty(readBufferSizeStr)) { - long readBufferSize = validateSizeString(readBufferSizeStr); - if (positiveCheck("ReadBufferSize", readBufferSize) - && maxValueCheck("ReadBufferSize", readBufferSize, Integer.MAX_VALUE)) { - readBufferSizeInBytes = Long.valueOf(readBufferSize).intValue(); - } - } - - Long offloadAfterElapsedInMillis = OffloadPoliciesImpl.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS; - if (StringUtils.isNotEmpty(offloadAfterElapsedStr)) { - Long offloadAfterElapsed; - try { - offloadAfterElapsed = TimeUnit.SECONDS.toMillis( - RelativeTimeUtil.parseRelativeTimeInSeconds(offloadAfterElapsedStr)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (maxValueCheck("OffloadAfterElapsed", offloadAfterElapsed, Long.MAX_VALUE)) { - offloadAfterElapsedInMillis = offloadAfterElapsed; - } - } - - Long offloadAfterThresholdInBytes = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES; - if (StringUtils.isNotEmpty(offloadAfterThresholdStr)) { - long offloadAfterThreshold = validateSizeString(offloadAfterThresholdStr); - if (maxValueCheck("OffloadAfterThreshold", offloadAfterThreshold, Long.MAX_VALUE)) { - offloadAfterThresholdInBytes = offloadAfterThreshold; - } - } - - Long offloadThresholdInSeconds = OffloadPoliciesImpl.DEFAULT_OFFLOAD_THRESHOLD_IN_SECONDS; - if (StringUtils.isNotEmpty(offloadAfterThresholdInSecondsStr)) { - Long offloadThresholdInSeconds0; - try { - offloadThresholdInSeconds0 = TimeUnit.SECONDS.toSeconds( - RelativeTimeUtil.parseRelativeTimeInSeconds(offloadAfterThresholdInSecondsStr.trim())); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (maxValueCheck("OffloadAfterThresholdInSeconds", offloadThresholdInSeconds0, Long.MAX_VALUE)) { - offloadThresholdInSeconds = offloadThresholdInSeconds0; - } - } - OffloadedReadPriority offloadedReadPriority = OffloadPoliciesImpl.DEFAULT_OFFLOADED_READ_PRIORITY; - if (this.offloadReadPriorityStr != null) { try { offloadedReadPriority = OffloadedReadPriority.fromString(this.offloadReadPriorityStr); @@ -2574,13 +2458,13 @@ private class SetCompactionThreshold extends CliCommand { @Parameter(names = { "--threshold", "-t" }, description = "Maximum number of bytes in a topic backlog before compaction is triggered " + "(eg: 10M, 16G, 3T). 0 disables automatic compaction", - required = true) - private String thresholdStr = "0"; + required = true, + converter = ByteUnitToLongConverter.class) + private Long threshold = 0L; @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long threshold = validateSizeString(thresholdStr); getTopics().setCompactionThreshold(persistentTopic, threshold); } } @@ -2957,8 +2841,9 @@ private class SetInactiveTopicPolicies extends CliCommand { @Parameter(names = {"--max-inactive-duration", "-t"}, description = "Max duration of topic inactivity " + "in seconds, topics that are inactive for longer than this value will be deleted " - + "(eg: 1s, 10s, 1m, 5h, 3d)", required = true) - private String deleteInactiveTopicsMaxInactiveDuration; + + "(eg: 1s, 10s, 1m, 5h, 3d)", required = true, + converter = TimeUnitToSecondsConverter.class) + private Long maxInactiveDurationInSeconds; @Parameter(names = { "--delete-mode", "-m" }, description = "Mode of delete inactive topic, Valid options are: " + "[delete_when_no_subscriptions, delete_when_subscriptions_caught_up]", required = true) @@ -2967,14 +2852,6 @@ private class SetInactiveTopicPolicies extends CliCommand { @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - long maxInactiveDurationInSeconds; - try { - maxInactiveDurationInSeconds = TimeUnit.SECONDS.toSeconds( - RelativeTimeUtil.parseRelativeTimeInSeconds(deleteInactiveTopicsMaxInactiveDuration)); - } catch (IllegalArgumentException exception) { - throw new ParameterException(exception.getMessage()); - } - if (enableDeleteWhileInactive == disableDeleteWhileInactive) { throw new ParameterException("Need to specify either enable-delete-while-inactive " + "or disable-delete-while-inactive"); @@ -2987,7 +2864,7 @@ void run() throws PulsarAdminException { + "or delete_when_subscriptions_caught_up"); } getTopics().setInactiveTopicPolicies(persistentTopic, new InactiveTopicPolicies(deleteMode, - (int) maxInactiveDurationInSeconds, enableDeleteWhileInactive)); + maxInactiveDurationInSeconds.intValue(), enableDeleteWhileInactive)); } } From 39235edcb45a615627dcd0471ed0872e568790ff Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 16 Oct 2023 15:47:16 +0300 Subject: [PATCH 15/37] [fix][test] Fix a resource leak in ClusterMigrationTest (#21366) --- .../broker/service/ClusterMigrationTest.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) 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 b1fd11ba0ac42..a2f32170f0f88 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 @@ -25,13 +25,13 @@ 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.lang.reflect.Method; import java.net.URL; import java.util.List; import java.util.Optional; import java.util.concurrent.TimeUnit; - +import lombok.Cleanup; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -55,10 +55,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.Sets; - -import lombok.Cleanup; - @Test(groups = "broker") public class ClusterMigrationTest { @@ -232,9 +228,13 @@ public void setup() throws Exception { protected void cleanup() throws Exception { log.info("--- Shutting down ---"); broker1.cleanup(); + admin1.close(); broker2.cleanup(); + admin2.close(); broker3.cleanup(); + admin3.close(); broker4.cleanup(); + admin4.close(); } @BeforeMethod(alwaysRun = true) @@ -459,7 +459,7 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc assertEquals(topic1.getReplicators().size(), 1); // stop service in the replication cluster to build replication backlog - broker3.cleanup(); + broker3.stop(); retryStrategically((test) -> broker3.getPulsarService() == null, 10, 1000); assertNull(pulsar3.getBrokerService()); @@ -529,7 +529,7 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc /** * This test validates that blue cluster first creates list of subscriptions into green cluster so, green cluster * will not lose the data if producer migrates. - * + * * @throws Exception */ @Test @@ -928,7 +928,7 @@ public void testNamespaceMigrationWithReplicationBacklog(boolean persistent, Sub assertEquals(blueTopicNs2.getReplicators().size(), 1); // stop service in the replication cluster to build replication backlog - broker3.cleanup(); + broker3.stop(); retryStrategically((test) -> broker3.getPulsarService() == null, 10, 1000); assertNull(pulsar3.getBrokerService()); @@ -1063,9 +1063,13 @@ public String getClusterName() { return configClusterName; } + public void stop() throws Exception { + stopBroker(); + } + @Override protected void cleanup() throws Exception { - stopBroker(); + internalCleanup(); } public void restart() throws Exception { From e08841114cece7e4dc64677505ab66271a2bace5 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 16 Oct 2023 11:26:29 -0700 Subject: [PATCH 16/37] [fix][broker] Fix avoid creating new topic after migration is started (#21368) Co-authored-by: Rajan Dhabalia --- .../pulsar/broker/service/AbstractTopic.java | 5 ++ .../pulsar/broker/service/BrokerService.java | 26 +++++++++- .../pulsar/broker/service/ServerCnx.java | 51 +++++++++++++++++-- .../broker/service/ClusterMigrationTest.java | 43 ++++++++++++++-- 4 files changed, 114 insertions(+), 11 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 3cb396d7a4b41..a8f25f61a9451 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 @@ -1352,6 +1352,11 @@ public Optional getMigratedClusterUrl() { return getMigratedClusterUrl(brokerService.getPulsar(), topic); } + public static CompletableFuture isClusterMigrationEnabled(PulsarService pulsar, + String topic) { + return getMigratedClusterUrlAsync(pulsar, topic).thenApply(url -> url.isPresent()); + } + public static CompletableFuture> getMigratedClusterUrlAsync(PulsarService pulsar, String topic) { return pulsar.getPulsarResources().getClusterResources().getClusterAsync(pulsar.getConfig().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 d03a94a0563ab..b85f77cb2f59c 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 @@ -111,6 +111,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; +import org.apache.pulsar.broker.service.BrokerServiceException.TopicMigratedException; import org.apache.pulsar.broker.service.TopicEventsListener.EventStage; import org.apache.pulsar.broker.service.TopicEventsListener.TopicEvent; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; @@ -1521,6 +1522,12 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S topicFuture.handle((persistentTopic, ex) -> { // release permit and process pending topic topicLoadSemaphore.release(); + // 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)) { + topicFuture.completeExceptionally(ex.getCause()); + return null; + } createPendingLoadTopic(); return null; }); @@ -1632,7 +1639,10 @@ private void createPersistentTopic(final String topic, boolean createIfMissing, ? checkMaxTopicsPerNamespace(topicName, 1) : CompletableFuture.completedFuture(null); - maxTopicsCheck.thenCompose(__ -> getManagedLedgerConfig(topicName)).thenAccept(managedLedgerConfig -> { + CompletableFuture isTopicAlreadyMigrated = checkTopicAlreadyMigrated(topicName); + + maxTopicsCheck.thenCompose(__ -> isTopicAlreadyMigrated).thenCompose(__ -> getManagedLedgerConfig(topicName)) + .thenAccept(managedLedgerConfig -> { if (isBrokerEntryMetadataEnabled() || isBrokerPayloadProcessorEnabled()) { // init managedLedger interceptor Set interceptors = new HashSet<>(); @@ -1760,6 +1770,20 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { }); } + private CompletableFuture checkTopicAlreadyMigrated(TopicName topicName) { + CompletableFuture result = new CompletableFuture<>(); + AbstractTopic.isClusterMigrationEnabled(pulsar, topicName.toString()).handle((isMigrated, ex) -> { + if (isMigrated) { + result.completeExceptionally( + new BrokerServiceException.TopicMigratedException(topicName + " already migrated")); + } else { + result.complete(null); + } + return null; + }); + return result; + } + public CompletableFuture getManagedLedgerConfig(@Nonnull TopicName topicName) { requireNonNull(topicName); NamespaceName namespace = topicName.getNamespaceObject(); 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 18bf62f1ef34a..efbe3fcd7ec6d 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 @@ -1304,6 +1304,24 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { remoteAddress, topicName, subscriptionName, exception.getCause().getMessage()); } + } else if (exception.getCause() instanceof BrokerServiceException.TopicMigratedException) { + Optional clusterURL = getMigratedClusterUrl(service.getPulsar(), + topicName.toString()); + if (clusterURL.isPresent()) { + log.info("[{}] redirect migrated consumer to topic {}: " + + "consumerId={}, subName={}, {}", remoteAddress, + topicName, consumerId, subscriptionName, exception.getCause().getMessage()); + boolean msgSent = commandSender.sendTopicMigrated(ResourceType.Consumer, consumerId, + clusterURL.get().getBrokerServiceUrl(), + clusterURL.get().getBrokerServiceUrlTls()); + if (!msgSent) { + log.info("consumer client doesn't support topic migration handling {}-{}-{}", + topicName, remoteAddress, consumerId); + } + consumers.remove(consumerId, consumerFuture); + closeConsumer(consumerId); + return null; + } } else if (exception.getCause() instanceof BrokerServiceException) { log.warn("[{}][{}][{}] Failed to create consumer: consumerId={}, {}", remoteAddress, topicName, subscriptionName, @@ -1567,6 +1585,22 @@ protected void handleProducer(final CommandProducer cmdProducer) { } producers.remove(producerId, producerFuture); return null; + } else if (cause instanceof BrokerServiceException.TopicMigratedException) { + Optional clusterURL = getMigratedClusterUrl(service.getPulsar(), topicName.toString()); + if (clusterURL.isPresent()) { + log.info("[{}] redirect migrated producer to topic {}: " + + "producerId={}, producerName = {}, {}", remoteAddress, + topicName, producerId, producerName, cause.getMessage()); + boolean msgSent = commandSender.sendTopicMigrated(ResourceType.Producer, producerId, + clusterURL.get().getBrokerServiceUrl(), clusterURL.get().getBrokerServiceUrlTls()); + if (!msgSent) { + log.info("client doesn't support topic migration handling {}-{}-{}", topicName, + remoteAddress, producerId); + } + producers.remove(producerId, producerFuture); + closeProducer(producerId, -1L); + return null; + } } // Do not print stack traces for expected exceptions @@ -2986,15 +3020,18 @@ protected void interceptCommand(BaseCommand command) throws InterceptException { public void closeProducer(Producer producer) { // removes producer-connection from map and send close command to producer safelyRemoveProducer(producer); + closeProducer(producer.getProducerId(), producer.getEpoch()); + + } + + public void closeProducer(long producerId, long epoch) { if (getRemoteEndpointProtocolVersion() >= v5.getValue()) { - writeAndFlush(Commands.newCloseProducer(producer.getProducerId(), -1L)); + 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 // because they are expected. Once the interval has passed, the client should have received the // CloseProducer command and should not send any additional messages until it sends a create Producer // command. - final long epoch = producer.getEpoch(); - final long producerId = producer.getProducerId(); recentlyClosedProducers.put(producerId, epoch); ctx.executor().schedule(() -> { recentlyClosedProducers.remove(producerId, epoch); @@ -3009,8 +3046,12 @@ public void closeProducer(Producer producer) { public void closeConsumer(Consumer consumer) { // removes consumer-connection from map and send close command to consumer safelyRemoveConsumer(consumer); + closeConsumer(consumer.consumerId()); + } + + public void closeConsumer(long consumerId) { if (getRemoteEndpointProtocolVersion() >= v5.getValue()) { - writeAndFlush(Commands.newCloseConsumer(consumer.consumerId(), -1L)); + writeAndFlush(Commands.newCloseConsumer(consumerId, -1L)); } else { close(); } @@ -3529,4 +3570,4 @@ protected AuthenticationState getOriginalAuthState() { protected void setAuthRole(String authRole) { this.authRole = authRole; } -} +} \ No newline at end of file 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 a2f32170f0f88..f2e0cf7fd12fb 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 @@ -368,8 +368,8 @@ public void testClusterMigration(boolean persistent, SubscriptionType subType) t String newTopicName = topicName + "-new"; consumerM = client1.newConsumer().topic(newTopicName).subscriptionType(subType) .subscriptionName("sM").subscribe(); - retryStrategically((t) -> pulsar1.getBrokerService().getTopicReference(newTopicName).isPresent(), 5, 100); - pulsar1.getBrokerService().getTopicReference(newTopicName).get().checkClusterMigration().get(); + retryStrategically((t) -> pulsar2.getBrokerService().getTopicReference(newTopicName).isPresent(), 5, 100); + pulsar2.getBrokerService().getTopicReference(newTopicName).get().checkClusterMigration().get(); retryStrategically((t) -> pulsar2.getBrokerService().getTopicReference(newTopicName).isPresent() && pulsar2.getBrokerService().getTopicReference(newTopicName).get().getSubscription("sM") @@ -398,7 +398,7 @@ public void testClusterMigration(boolean persistent, SubscriptionType subType) t .subscriptionName("s1-d").subscribe(); Producer producerDiff = client1.newProducer().topic(diffTopic).enableBatching(false) .producerName("cluster1-d").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - AbstractTopic topicDiff = (AbstractTopic) pulsar1.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); + AbstractTopic topicDiff = (AbstractTopic) pulsar2.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); assertNotNull(topicDiff); for (int i = 0; i < n; i++) { producerDiff.send("diff".getBytes()); @@ -603,6 +603,39 @@ public void testClusterMigrationWithResourceCreated() throws Exception { consumer1.close(); producer1.close(); + + // publish to new topic which should be redirected immediately + String newTopic = topicName+"-new"; + producer1 = client1.newProducer().topic(newTopic).enableBatching(false) + .producerName("cluster1-1").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + retryStrategically((test) -> { + try { + pulsar2.getBrokerService().getTopic(newTopic, false).getNow(null).get(); + return true; + } catch (Exception e) { + // ok + } + return false; + }, 10, 500); + PersistentTopic pulsar2Topic = (PersistentTopic) pulsar2.getBrokerService().getTopic(newTopic, false).getNow(null) + .get(); + retryStrategically((test) -> { + try { + return !pulsar2Topic.getProducers().isEmpty(); + } catch (Exception e) { + return false; + } + }, 10, 500); + assertFalse(pulsar2Topic.getProducers().isEmpty()); + consumer1 = client1.newConsumer().topic(newTopic).subscriptionName("s1").subscribe(); + retryStrategically((test) -> { + try { + return !pulsar2Topic.getSubscription("s1").getConsumers().isEmpty(); + } catch (Exception e) { + return false; + } + }, 10, 500); + assertFalse(pulsar2Topic.getSubscription("s1").getConsumers().isEmpty()); } @Test(dataProvider = "NamespaceMigrationTopicSubscriptionTypes") @@ -803,7 +836,7 @@ public void testNamespaceMigration(boolean persistent, SubscriptionType subType, consumerM = client1.newConsumer().topic(newTopicName).subscriptionType(subType) .subscriptionName("sM").subscribe(); retryStrategically((t) -> pulsar1.getBrokerService().getTopicReference(newTopicName).isPresent(), 5, 100); - pulsar1.getBrokerService().getTopicReference(newTopicName).get().checkClusterMigration().get(); + pulsar2.getBrokerService().getTopicReference(newTopicName).get().checkClusterMigration().get(); retryStrategically((t) -> pulsar2.getBrokerService().getTopicReference(newTopicName).isPresent() && pulsar2.getBrokerService().getTopicReference(newTopicName).get().getSubscription("sM") @@ -832,7 +865,7 @@ public void testNamespaceMigration(boolean persistent, SubscriptionType subType, .subscriptionName("s1-d").subscribe(); Producer producerDiff = client1.newProducer().topic(diffTopic).enableBatching(false) .producerName("cluster1-d").messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - AbstractTopic topicDiff = (AbstractTopic) pulsar1.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); + AbstractTopic topicDiff = (AbstractTopic) pulsar2.getBrokerService().getTopic(diffTopic, false).getNow(null).get(); assertNotNull(topicDiff); for (int i = 0; i < n; i++) { producerDiff.send("diff".getBytes()); From 689976bf3ea8dc4cff6f4fdcab54bdd4a8a73a7e Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 16 Oct 2023 14:05:13 -0700 Subject: [PATCH 17/37] [fix][broker] Fix race condition of replication cluster connection during migration topic (#21364) Co-authored-by: Rajan Dhabalia --- .../pulsar/broker/service/ServerCnx.java | 2 +- .../apache/pulsar/broker/service/Topic.java | 2 + .../nonpersistent/NonPersistentTopic.java | 5 +++ .../service/persistent/PersistentTopic.java | 37 +++++++++++++++++-- .../broker/service/ClusterMigrationTest.java | 9 ++++- 5 files changed, 49 insertions(+), 6 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 efbe3fcd7ec6d..95f139dc11e43 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 @@ -1680,7 +1680,7 @@ private void buildProducerAndAddTopic(Topic topic, long producerId, String produ if (ex.getCause() instanceof BrokerServiceException.TopicMigratedException) { Optional clusterURL = getMigratedClusterUrl(service.getPulsar(), topic.getName()); if (clusterURL.isPresent()) { - if (topic.isReplicationBacklogExist()) { + if (!topic.shouldProducerMigrate()) { log.info("Topic {} is migrated but replication backlog exist: " + "producerId = {}, producerName = {}, {}", topicName, producerId, producerName, ex.getCause().getMessage()); 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 7657d77e1299f..c697639ff4fa1 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 @@ -234,6 +234,8 @@ CompletableFuture createSubscription(String subscriptionName, Init boolean isBrokerPublishRateExceeded(); + boolean shouldProducerMigrate(); + boolean isReplicationBacklogExist(); void disableCnxAutoRead(); 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 54811da723808..76e9f261ca6a9 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 @@ -235,6 +235,11 @@ public void checkMessageDeduplicationInfo() { // No-op } + @Override + public boolean shouldProducerMigrate() { + return true; + } + @Override public boolean isReplicationBacklogExist() { return false; 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 1ed7bafc14724..03ee0f06e2fbb 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 @@ -660,8 +660,9 @@ public synchronized void addFailed(ManagedLedgerException exception, Object ctx) List> futures = new ArrayList<>(); // send migration url metadata to producers before disconnecting them if (isMigrated()) { - if (isReplicationBacklogExist()) { - log.info("Topic {} is migrated but replication backlog exists. Closing producers.", topic); + if (!shouldProducerMigrate()) { + log.info("Topic {} is migrated but replication-backlog exists or " + + "subs not created. Closing producers.", topic); } else { producers.forEach((__, producer) -> producer.topicMigrated(getMigratedClusterUrl())); } @@ -2592,6 +2593,20 @@ public CompletableFuture checkClusterMigration() { if (!clusterUrl.isPresent()) { return CompletableFuture.completedFuture(null); } + + if (isReplicated()) { + if (isReplicationBacklogExist()) { + if (!ledger.isMigrated()) { + log.info("{} applying migration with replication backlog", topic); + ledger.asyncMigrate(); + } + if (log.isDebugEnabled()) { + log.debug("{} has replication backlog and applied migraiton", topic); + } + return CompletableFuture.completedFuture(null); + } + } + return initMigration().thenCompose(subCreated -> { migrationSubsCreated = true; CompletableFuture migrated = !isMigrated() ? ledger.asyncMigrate() @@ -2603,7 +2618,9 @@ public CompletableFuture checkClusterMigration() { } }); return null; - }).thenCompose(__ -> checkAndDisconnectReplicators()).thenCompose(__ -> checkAndUnsubscribeSubscriptions()); + }).thenCompose(__ -> checkAndDisconnectReplicators()) + .thenCompose(__ -> checkAndUnsubscribeSubscriptions()) + .thenCompose(__ -> checkAndDisconnectProducers()); }); } @@ -2710,6 +2727,15 @@ private CompletableFuture checkAndUnsubscribeSubscriptions() { return FutureUtil.waitForAll(futures); } + private CompletableFuture checkAndDisconnectProducers() { + List> futures = new ArrayList<>(); + producers.forEach((name, producer) -> { + futures.add(producer.disconnect()); + }); + + return FutureUtil.waitForAll(futures); + } + private CompletableFuture checkAndDisconnectReplicators() { List> futures = new ArrayList<>(); ConcurrentOpenHashMap replicators = getReplicators(); @@ -2721,6 +2747,11 @@ private CompletableFuture checkAndDisconnectReplicators() { return FutureUtil.waitForAll(futures); } + public boolean shouldProducerMigrate() { + return !isReplicationBacklogExist() && migrationSubsCreated; + } + + @Override public boolean isReplicationBacklogExist() { ConcurrentOpenHashMap replicators = getReplicators(); if (replicators != null) { 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 f2e0cf7fd12fb..aa5444edc9901 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 @@ -309,7 +309,6 @@ public void testClusterMigration(boolean persistent, SubscriptionType subType) t return false; }, 10, 500); - topic1.checkClusterMigration().get(); log.info("before sending message"); @@ -521,8 +520,11 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc retryStrategically((test) -> !topic1.isReplicationBacklogExist(), 10, 1000); assertFalse(topic1.isReplicationBacklogExist()); - producer1.send("test".getBytes()); // verify that the producer1 is now connected to migrated cluster "r2" since backlog is cleared. + topic1.checkClusterMigration().get(); + + // verify that the producer1 is now is now connected to migrated cluster "r2" since backlog is cleared. + retryStrategically((test) -> topic2.getProducers().size()==2, 10, 500); assertEquals(topic2.getProducers().size(), 2); } @@ -1046,6 +1048,9 @@ public void testNamespaceMigrationWithReplicationBacklog(boolean persistent, Sub retryStrategically((test) -> !blueTopicNs2.isReplicationBacklogExist(), 10, 1000); assertFalse(blueTopicNs2.isReplicationBacklogExist()); + blueTopicNs1.checkClusterMigration().get(); + blueTopicNs2.checkClusterMigration().get(); + // verify that the producer1 is now is now connected to migrated cluster green since backlog is cleared. retryStrategically((test) -> greenTopicNs1.getProducers().size()==2, 10, 500); assertEquals(greenTopicNs1.getProducers().size(), 2); From f5222d6b1f64d14029f8fef3e0108cf74ffefded Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 17 Oct 2023 09:38:02 +0800 Subject: [PATCH 18/37] [fix][sec] Bump avro version to 1.11.3 for CVE-2023-39410 (#21341) Signed-off-by: tison --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- .../schema/compatibility/SchemaCompatibilityCheckTest.java | 2 +- .../pulsar/client/impl/schema/ProtobufSchemaTest.java | 6 +++--- pulsar-io/kafka-connect-adaptor/pom.xml | 6 ++++++ pulsar-sql/presto-distribution/LICENSE | 4 ++-- 7 files changed, 17 insertions(+), 11 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 42033316b717f..dcad8e9bb78f9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -447,8 +447,8 @@ The Apache Software License, Version 2.0 - net.jodah-typetools-0.5.0.jar - net.jodah-failsafe-2.4.4.jar * Apache Avro - - org.apache.avro-avro-1.10.2.jar - - org.apache.avro-avro-protobuf-1.10.2.jar + - org.apache.avro-avro-1.11.3.jar + - org.apache.avro-avro-protobuf-1.11.3.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 9db4713f8e4da..8a8f47350409c 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -407,8 +407,8 @@ The Apache Software License, Version 2.0 * Google Error Prone Annotations - error_prone_annotations-2.5.1.jar * Javassist -- javassist-3.25.0-GA.jar * Apache Avro - - avro-1.10.2.jar - - avro-protobuf-1.10.2.jar + - avro-1.11.3.jar + - avro-protobuf-1.11.3.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 9b7e256addfa4..2ac82aaee7618 100644 --- a/pom.xml +++ b/pom.xml @@ -177,7 +177,7 @@ flexible messaging model and an intuitive client API. 3.4.0 5.5.3 1.12.262 - 1.10.2 + 1.11.3 2.10.10 2.5.0 5.1.0 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java index 140dea9e7ebc7..49517a424b936 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java @@ -407,7 +407,7 @@ public void testSchemaComparison() throws Exception { assertEquals(admin.namespaces().getSchemaCompatibilityStrategy(namespaceName.toString()), SchemaCompatibilityStrategy.UNDEFINED); byte[] changeSchemaBytes = (new String(Schema.AVRO(Schemas.PersonOne.class) - .getSchemaInfo().getSchema(), UTF_8) + "/n /n /n").getBytes(); + .getSchemaInfo().getSchema(), UTF_8) + "\n \n \n").getBytes(); SchemaInfo schemaInfo = SchemaInfo.builder().type(SchemaType.AVRO).schema(changeSchemaBytes).build(); admin.schemas().createSchema(fqtn, schemaInfo); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java index 3fcd6f12b982d..85012276d5af1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java @@ -41,20 +41,20 @@ public class ProtobufSchemaTest { "\"namespace\":\"org.apache.pulsar.client.schema.proto.Test\"," + "\"fields\":[{\"name\":\"stringField\",\"type\":{\"type\":\"string\"," + "\"avro.java.string\":\"String\"},\"default\":\"\"},{\"name\":\"doubleField\"," + - "\"type\":\"double\",\"default\":0},{\"name\":\"intField\",\"type\":\"int\"," + + "\"type\":\"double\",\"default\":0.0},{\"name\":\"intField\",\"type\":\"int\"," + "\"default\":0},{\"name\":\"testEnum\",\"type\":{\"type\":\"enum\"," + "\"name\":\"TestEnum\",\"symbols\":[\"SHARED\",\"FAILOVER\"]}," + "\"default\":\"SHARED\"},{\"name\":\"nestedField\"," + "\"type\":[\"null\",{\"type\":\"record\",\"name\":\"SubMessage\"," + "\"fields\":[{\"name\":\"foo\",\"type\":{\"type\":\"string\"," + "\"avro.java.string\":\"String\"},\"default\":\"\"}" + - ",{\"name\":\"bar\",\"type\":\"double\",\"default\":0}]}]" + + ",{\"name\":\"bar\",\"type\":\"double\",\"default\":0.0}]}]" + ",\"default\":null},{\"name\":\"repeatedField\",\"type\":{\"type\":\"array\"" + ",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},\"default\":[]}" + ",{\"name\":\"externalMessage\",\"type\":[\"null\",{\"type\":\"record\"" + ",\"name\":\"ExternalMessage\",\"namespace\":\"org.apache.pulsar.client.schema.proto.ExternalTest\"" + ",\"fields\":[{\"name\":\"stringField\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}," + - "\"default\":\"\"},{\"name\":\"doubleField\",\"type\":\"double\",\"default\":0}]}],\"default\":null}]}"; + "\"default\":\"\"},{\"name\":\"doubleField\",\"type\":\"double\",\"default\":0.0}]}],\"default\":null}]}"; private static final String EXPECTED_PARSING_INFO = "{\"__alwaysAllowNull\":\"true\",\"__jsr310ConversionEnabled\":\"false\"," + "\"__PARSING_INFO__\":\"[{\\\"number\\\":1,\\\"name\\\":\\\"stringField\\\",\\\"type\\\":\\\"STRING\\\"," + diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 7b49b337f6c03..131cf605cb7bd 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -116,6 +116,12 @@ io.confluent kafka-connect-avro-converter ${confluent.version} + + + org.apache.avro + avro + + diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index aa2577776619e..7c3f0f70cf228 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -372,8 +372,8 @@ The Apache Software License, Version 2.0 * OpenCSV - opencsv-2.3.jar * Avro - - avro-1.10.2.jar - - avro-protobuf-1.10.2.jar + - avro-1.11.3.jar + - avro-protobuf-1.11.3.jar * Caffeine - caffeine-2.9.1.jar * Javax From e6cd11fb49f2d8f3376f2218e60f8c5909f66348 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Oct 2023 13:17:58 +0300 Subject: [PATCH 19/37] [fix][test] Fix some resource leaks in compaction tests (#21374) --- .../java/org/apache/pulsar/compaction/CompactedTopicTest.java | 3 +++ .../pulsar/compaction/ServiceUnitStateCompactionTest.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) 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 4a098fd0eb9f0..33e4e998ad6cf 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 @@ -161,6 +161,7 @@ public void cleanup() throws Exception { @Test public void testEntryLookup() throws Exception { + @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null); @@ -216,6 +217,7 @@ public void testEntryLookup() throws Exception { @Test public void testCleanupOldCompactedTopicLedger() throws Exception { + @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null); @@ -845,6 +847,7 @@ public void testReadCompactedLatestMessageWithInclusive() throws Exception { @Test public void testCompactWithConcurrentGetCompactionHorizonAndCompactedTopicContext() throws Exception { + @Cleanup BookKeeper bk0 = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null); 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 e4f0750a981c9..09c3ebe419394 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 @@ -168,7 +168,7 @@ public void setup() throws Exception { @Override public void cleanup() throws Exception { super.internalCleanup(); - + bk.close(); if (compactionScheduler != null) { compactionScheduler.shutdownNow(); } From ffc083b5ea9da998c35161b9dcfbfb6e38e3917e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Oct 2023 15:34:16 +0300 Subject: [PATCH 20/37] [fix][test] Fix resource leaks with Pulsar Functions tests (#21378) --- .../pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java | 1 - .../pulsar/functions/worker/PulsarFunctionPublishTest.java | 1 - .../pulsar/functions/worker/PulsarWorkerAssignmentTest.java | 1 - .../test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java | 1 - .../test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java | 1 - 5 files changed, 5 deletions(-) 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 714c9d7269970..107aedd076691 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 @@ -268,7 +268,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationProvider(config.getAuthorizationProvider()); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } 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 c820f512a68de..7bcf1dec871e0 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 @@ -269,7 +269,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationEnabled(true); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } 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 0821974bea506..6226fa904885c 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 @@ -174,7 +174,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setTopicCompactionFrequencySec(1); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } 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 3a99cc647ed5c..f968315a7124c 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 @@ -307,7 +307,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationEnabled(true); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } } 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 d31d0c66bdf93..22b9ad0df3a69 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 @@ -211,7 +211,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setTlsTrustCertsFilePath(TLS_CLIENT_CERT_FILE_PATH); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } } From cb7c98a7f56c3115a9e73bff96b8b4daac2c8180 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Oct 2023 21:50:16 +0300 Subject: [PATCH 21/37] [fix][test] Fix resource leak in TransactionCoordinatorClientTest (#21380) --- .../coordinator/TransactionMetaStoreTestBase.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 eb714dd848afc..7a0fb48f91150 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 @@ -120,6 +120,9 @@ public final void shutdownAll() throws Exception { @Override protected void cleanup() throws Exception { + if (transactionCoordinatorClient != null) { + transactionCoordinatorClient.close(); + } for (PulsarAdmin admin : pulsarAdmins) { if (admin != null) { admin.close(); @@ -133,6 +136,9 @@ protected void cleanup() throws Exception { service.close(); } } + if (bkEnsemble != null) { + bkEnsemble.stop(); + } Mockito.reset(); } } From de8f543fb9be71208578f35fcb397e7db3482107 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 03:05:52 +0300 Subject: [PATCH 22/37] [fix][test] Fix resource leak in PulsarServiceTest (#21386) --- .../apache/pulsar/broker/PulsarServiceTest.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) 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 3e0887646e119..daa4393db55fd 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 @@ -188,7 +188,7 @@ public void testDynamicBrokerPort() throws Exception { } @Test - public void testBacklogAndRetentionCheck() { + public void testBacklogAndRetentionCheck() throws PulsarServerException { ServiceConfiguration config = new ServiceConfiguration(); config.setClusterName("test"); config.setMetadataStoreUrl("memory:local"); @@ -200,6 +200,8 @@ public void testBacklogAndRetentionCheck() { pulsarService.start(); } catch (Exception e) { assertFalse(e.getCause() instanceof IllegalArgumentException); + } finally { + pulsarService.close(); } // Only set retention @@ -212,6 +214,8 @@ public void testBacklogAndRetentionCheck() { pulsarService.start(); } catch (Exception e) { assertFalse(e.getCause() instanceof IllegalArgumentException); + } finally { + pulsarService.close(); } // Set both retention and backlog quota @@ -224,6 +228,8 @@ public void testBacklogAndRetentionCheck() { pulsarService.start(); } catch (Exception e) { assertFalse(e.getCause() instanceof IllegalArgumentException); + } finally { + pulsarService.close(); } // Set invalidated retention and backlog quota @@ -235,6 +241,8 @@ public void testBacklogAndRetentionCheck() { pulsarService.start(); } catch (Exception e) { assertTrue(e.getCause() instanceof IllegalArgumentException); + } finally { + pulsarService.close(); } config.setBacklogQuotaDefaultLimitBytes(4 * 1024 * 1024); @@ -246,6 +254,8 @@ public void testBacklogAndRetentionCheck() { pulsarService.start(); } catch (Exception e) { assertTrue(e.getCause() instanceof IllegalArgumentException); + } finally { + pulsarService.close(); } // Only set backlog quota @@ -258,6 +268,8 @@ public void testBacklogAndRetentionCheck() { pulsarService.start(); } catch (Exception e) { assertFalse(e.getCause() instanceof IllegalArgumentException); + } finally { + pulsarService.close(); } } } From da7e5458c1eb245ba30d4a65422df19f1b0ae4b2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 03:06:03 +0300 Subject: [PATCH 23/37] [fix][test] Cleanup cached executors in tests (#21383) --- .../common/util/collections/ConcurrentLongHashMapTest.java | 1 + .../util/collections/ConcurrentLongLongPairHashMapTest.java | 4 ++++ .../common/util/collections/ConcurrentLongPairSetTest.java | 1 + .../common/util/collections/ConcurrentOpenHashMapTest.java | 1 + .../common/util/collections/ConcurrentOpenHashSetTest.java | 1 + .../pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java | 1 + 6 files changed, 9 insertions(+) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java index e1f947ad8c4f6..60bfbd31c2868 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java @@ -224,6 +224,7 @@ public void testConcurrentExpandAndShrinkAndGet() throws Throwable { .build(); assertEquals(map.capacity(), 4); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int readThreads = 16; final int writeThreads = 1; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java index 0de3fdb5c84bf..e2d7c77f51622 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java @@ -37,6 +37,7 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; +import lombok.Cleanup; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; import org.testng.annotations.Test; @@ -186,6 +187,7 @@ public void testConcurrentExpandAndShrinkAndGet() throws Throwable { .build(); assertEquals(map.capacity(), 4); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int readThreads = 16; final int writeThreads = 1; @@ -335,6 +337,7 @@ public void testRehashingWithDeletes() { public void concurrentInsertions() throws Throwable { ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder() .build(); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int nThreads = 16; @@ -375,6 +378,7 @@ public void concurrentInsertions() throws Throwable { public void concurrentInsertionsAndReads() throws Throwable { ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder() .build(); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int nThreads = 16; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java index bce2b8993835f..9c8655b21cfb3 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java @@ -223,6 +223,7 @@ public void testConcurrentExpandAndShrinkAndGet() throws Throwable { .build(); assertEquals(set.capacity(), 4); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int readThreads = 16; final int writeThreads = 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 index 410d490b98faa..8ff00a022b3cb 100644 --- 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 @@ -227,6 +227,7 @@ public void testConcurrentExpandAndShrinkAndGet() throws Throwable { .build(); assertEquals(map.capacity(), 4); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int readThreads = 16; final int writeThreads = 1; 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 index 6a40095ab0647..29722ed782127 100644 --- 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 @@ -198,6 +198,7 @@ public void testConcurrentExpandAndShrinkAndGet() throws Throwable { .build(); assertEquals(set.capacity(), 4); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final int readThreads = 16; final int writeThreads = 1; diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java index b64cc964a999c..f8a51602686ed 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/bookkeeper/LedgerManagerIteratorTest.java @@ -403,6 +403,7 @@ public void checkConcurrentModifications(String provider, Supplier urlSu final long start = MathUtils.nowInNano(); final CountDownLatch latch = new CountDownLatch(1); ArrayList> futures = new ArrayList<>(); + @Cleanup("shutdownNow") ExecutorService executor = Executors.newCachedThreadPool(); final ConcurrentSkipListSet createdLedgers = new ConcurrentSkipListSet<>(); for (int i = 0; i < numWriters; ++i) { From 602478f28eda402875beb0cd616a3a3262079a8e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 03:06:24 +0300 Subject: [PATCH 24/37] [fix][test] Cleanup test resources in pulsar-client module (#21382) --- .../pulsar/client/impl/AutoClusterFailoverTest.java | 7 +++++++ .../apache/pulsar/client/impl/ClientTestFixtures.java | 8 +++++++- .../client/impl/ControlledClusterFailoverTest.java | 3 +++ .../pulsar/client/impl/PartitionedProducerImplTest.java | 9 ++++++++- .../client/impl/ProducerStatsRecorderImplTest.java | 4 ++++ .../apache/pulsar/client/impl/PulsarClientImplTest.java | 1 + .../pulsar/client/impl/auth/AuthenticationTokenTest.java | 2 ++ .../client/impl/conf/ConfigurationDataUtilsTest.java | 3 +++ 8 files changed, 35 insertions(+), 2 deletions(-) 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 63fbb239439bd..545cf7483e4e3 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 @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; @@ -48,6 +49,7 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException long failoverDelay = 30; long switchBackDelay = 60; long checkInterval = 1_000; + @Cleanup ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -86,6 +88,7 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException Map secondaryAuthentications = new HashMap<>(); secondaryAuthentications.put(secondary, secondaryAuthentication); + @Cleanup ServiceUrlProvider provider1 = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -112,6 +115,7 @@ public void testInitialize() { ClientConfigurationData configurationData = new ClientConfigurationData(); + @Cleanup ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -156,6 +160,7 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { ClientConfigurationData configurationData = new ClientConfigurationData(); + @Cleanup ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -210,6 +215,7 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException configurationData.setTlsTrustCertsFilePath(primaryTlsTrustCertsFilePath); configurationData.setAuthentication(primaryAuthentication); + @Cleanup ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) @@ -265,6 +271,7 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { configurationData.setTlsTrustStorePath(primaryTlsTrustStorePath); configurationData.setTlsTrustStorePassword(primaryTlsTrustStorePassword); + @Cleanup ServiceUrlProvider provider = AutoClusterFailover.builder() .primary(primary) .secondary(Collections.singletonList(secondary)) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java index ff7d7f12dd452..738d969ac7449 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java @@ -24,6 +24,7 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.EventLoop; import io.netty.util.Timer; @@ -38,7 +39,12 @@ import org.mockito.Mockito; class ClientTestFixtures { - public static ScheduledExecutorService SCHEDULER = Executors.newSingleThreadScheduledExecutor(); + public static ScheduledExecutorService SCHEDULER = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("ClientTestFixtures-SCHEDULER-%d") + .setDaemon(true) + .build()); // static PulsarClientImpl createPulsarClientMock() { // return createPulsarClientMock(mock(ExecutorService.class)); 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 227e0db10b724..36160d40d540a 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 @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.asynchttpclient.Request; @@ -47,6 +48,7 @@ public void testBuildControlledClusterFailoverInstance() throws IOException { Map header = new HashMap<>(); header.put(keyA, valueA); header.put(keyB, valueB); + @Cleanup ServiceUrlProvider provider = ControlledClusterFailover.builder() .defaultServiceUrl(defaultServiceUrl) .urlProvider(urlProvider) @@ -81,6 +83,7 @@ public void testControlledClusterFailoverSwitch() throws IOException { controlledConfiguration.setAuthPluginClassName(authPluginClassName); controlledConfiguration.setAuthParamsString(authParamsString); + @Cleanup ServiceUrlProvider provider = ControlledClusterFailover.builder() .defaultServiceUrl(defaultServiceUrl) .urlProvider(urlProvider) 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 2bd18f69386f1..b38b17a731bea 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 @@ -29,10 +29,10 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.api.client.util.Lists; import io.netty.channel.EventLoopGroup; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; -import com.google.api.client.util.Lists; import java.lang.reflect.Field; import java.nio.ByteBuffer; import java.util.List; @@ -40,6 +40,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ThreadFactory; +import lombok.Cleanup; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRouter; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -188,8 +189,10 @@ public void testGetStats() throws Exception { conf.setStatsIntervalSeconds(100); ThreadFactory threadFactory = new DefaultThreadFactory("client-test-stats", Thread.currentThread().isDaemon()); + @Cleanup("shutdownGracefully") EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), false, threadFactory); + @Cleanup PulsarClientImpl clientImpl = new PulsarClientImpl(conf, eventLoopGroup); ProducerConfigurationData producerConfData = new ProducerConfigurationData(); @@ -214,9 +217,11 @@ public void testGetStatsWithoutArriveUpdateInterval() throws Exception { ThreadFactory threadFactory = new DefaultThreadFactory("client-test-stats", Thread.currentThread().isDaemon()); + @Cleanup("shutdownGracefully") EventLoopGroup eventLoopGroup = EventLoopUtil .newEventLoopGroup(conf.getNumIoThreads(), false, threadFactory); + @Cleanup PulsarClientImpl clientImpl = new PulsarClientImpl(conf, eventLoopGroup); ProducerConfigurationData producerConfData = new ProducerConfigurationData(); @@ -246,8 +251,10 @@ public void testGetNumOfPartitions() throws Exception { conf.setStatsIntervalSeconds(100); ThreadFactory threadFactory = new DefaultThreadFactory("client-test-stats", Thread.currentThread().isDaemon()); + @Cleanup("shutdownGracefully") EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), false, threadFactory); + @Cleanup PulsarClientImpl clientImpl = new PulsarClientImpl(conf, eventLoopGroup); ProducerConfigurationData producerConfData = new ProducerConfigurationData(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java index 27e2dcb37cee0..8f648bfd9ffbc 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java @@ -26,6 +26,7 @@ import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.testng.annotations.Test; @@ -43,6 +44,7 @@ public void testIncrementNumAcksReceived() throws Exception { ConnectionPool connectionPool = mock(ConnectionPool.class); when(client.getCnxPool()).thenReturn(connectionPool); when(client.getConfiguration()).thenReturn(conf); + @Cleanup("stop") Timer timer = new HashedWheelTimer(); when(client.timer()).thenReturn(timer); ProducerImpl producer = mock(ProducerImpl.class); @@ -55,6 +57,7 @@ public void testIncrementNumAcksReceived() throws Exception { recorder.incrementNumAcksReceived(latencyNs); Thread.sleep(1200); assertEquals(1000.0, recorder.getSendLatencyMillisMax(), 0.5); + recorder.cancelStatsTimeout(); } @Test @@ -65,6 +68,7 @@ public void testGetStatsAndCancelStatsTimeoutWithoutArriveUpdateInterval() { ConnectionPool connectionPool = mock(ConnectionPool.class); when(client.getCnxPool()).thenReturn(connectionPool); when(client.getConfiguration()).thenReturn(conf); + @Cleanup("stop") Timer timer = new HashedWheelTimer(); when(client.timer()).thenReturn(timer); ProducerImpl producer = mock(ProducerImpl.class); 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 e0b25db891247..c8278ccbd7a07 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 @@ -178,6 +178,7 @@ public void testInitializeWithoutTimer() throws Exception { @Test 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)); conf.setServiceUrl("pulsar://localhost:6650"); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationTokenTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationTokenTest.java index 589258eb09efb..a6e529d994031 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationTokenTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationTokenTest.java @@ -28,6 +28,7 @@ import java.nio.charset.StandardCharsets; import java.util.function.Supplier; +import lombok.Cleanup; import org.apache.commons.io.FileUtils; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; @@ -68,6 +69,7 @@ public void testAuthTokenClientConfig() throws Exception { clientConfig.setAuthentication(AuthenticationFactory.create( AuthenticationToken.class.getName(), "token-xyz")); + @Cleanup PulsarClientImpl pulsarClient = new PulsarClientImpl(clientConfig); Authentication authToken = pulsarClient.getConfiguration().getAuthentication(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConfigurationDataUtilsTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConfigurationDataUtilsTest.java index 354d25f5d7fe8..0e0117d400faa 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConfigurationDataUtilsTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ConfigurationDataUtilsTest.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableSet; +import lombok.Cleanup; import org.testng.Assert; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -148,6 +149,7 @@ public void testConfigBuilder() throws PulsarClientException { clientConfig.setServiceUrl("pulsar://unknown:6650"); clientConfig.setStatsIntervalSeconds(80); + @Cleanup PulsarClientImpl pulsarClient = new PulsarClientImpl(clientConfig); assertNotNull(pulsarClient, "Pulsar client built using config should not be null"); @@ -213,6 +215,7 @@ public void testSocks5() throws PulsarClientException { clientConfig.setSocks5ProxyUsername("test"); clientConfig.setSocks5ProxyPassword("test123"); + @Cleanup PulsarClientImpl pulsarClient = new PulsarClientImpl(clientConfig); assertEquals(pulsarClient.getConfiguration().getSocks5ProxyAddress(), new InetSocketAddress("localhost", 11080)); assertEquals(pulsarClient.getConfiguration().getSocks5ProxyUsername(), "test"); From 271906df8197f968e3655ec61e0cfbdb084d1487 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 03:25:56 +0300 Subject: [PATCH 25/37] [improve][ci] Upgrade Gradle Enterprise Maven extension (#21384) --- .mvn/extensions.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 872764f899827..de4bd7de54d9a 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,11 +24,11 @@ com.gradle gradle-enterprise-maven-extension - 1.17.1 + 1.19.3 com.gradle common-custom-user-data-maven-extension - 1.11.1 + 1.12.4 From 6681d330fa7a8d598f8c5bf0cd67111c11751b18 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 04:05:12 +0300 Subject: [PATCH 26/37] [fix][test] Fix resource leak in AdminApiTenantTest (#21387) --- .../org/apache/pulsar/broker/admin/AdminApiTenantTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTenantTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTenantTest.java index f883417614229..0cd9d9b737eba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTenantTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTenantTest.java @@ -21,7 +21,6 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; - import java.util.Collections; import java.util.List; import java.util.UUID; @@ -30,6 +29,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -46,7 +46,7 @@ public void setup() throws Exception { .createCluster(CLUSTER, ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); } - @BeforeClass(alwaysRun = true) + @AfterClass(alwaysRun = true) @Override public void cleanup() throws Exception { super.internalCleanup(); From 59422be479ce72ee34f7441df697011da115396a Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 17 Oct 2023 18:08:18 -0700 Subject: [PATCH 27/37] [improve][pip]PIP-307: Optimize Bundle Unload(Transfer) Protocol for ExtensibleLoadManager (#20748) --- pip/pip-307.md | 268 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 268 insertions(+) create mode 100644 pip/pip-307.md diff --git a/pip/pip-307.md b/pip/pip-307.md new file mode 100644 index 0000000000000..a919991d08991 --- /dev/null +++ b/pip/pip-307.md @@ -0,0 +1,268 @@ + + +# Background knowledge + +- Pulsar broker load balancer periodically unloads bundles from overloaded brokers. During this unload process, previous owner brokers close topic sessions(e.g. producers, subscriptions(consumers), managed ledgers). When re-assigned, new owner brokers recreate the topic sessions. + +- Pulsar clients request `CommandLookupTopic` to lookup or assign owner brokers for topics and connect to them. + +- PIP-192, the extensible load balancer introduced the bundle state channel that event-sources this unloading process in a state machine manner, from `releasing,` `assigned`, to `owned` state order. At `releasing,` the owner broker "releases" the bundle ownership(close topic sessions). + +- PIP-192, the extensible load balancer introduced TransferShedder, a new shedding strategy, which pre-assigns new owner brokers beforehand. + + +# Motivation + +- When unloading closes many topic sessions, then many clients need to request CommandLookupTopic at the same time, which could cause many lookup requests on brokers. This unloading process can be further optimized if we can let the client directly connect to the new owner broker without following `CommandLookupTopic` requests. +- In the new load balancer(pip-192), since the owner broker is already known, we can modify the close command protocol to pass the new destination broker URL and skip the lookup requests. +- Also, when unloading, we can gracefully shutdown ledgers -- we always close old managed ledgers first and then recreate it on the new owner without conflicts. + +# Goals +- Remove clients' lookup requests in the unload protocol to reduce the publish latency spike and e2e latency spike during +unloading and also to resolve bottlenecks (of thundering lookups) when there are a large number of topics in a cluster. +- Gracefully shutdown managed ledgers before new owners create them to reduce possible race-conditions between ledger close and ledger creations during unloading. + +## In Scope + + + +- This change will be added in the extensible load balancer. + +## Out of Scope + + + +- This won't change the existing load balancer behavior(modular load manager). + + + +# High Level Design + + + +To achieve the goals above, we could modify the bundle transfer protocol by the following. +The proposed protocol change is based on the bundle states from PIP-192. + +Basically, we could close the ledgers only in the releasing state and finally disconnect clients in the owned state with destination broker urls. The clients will directly connect to the pre-assigned destination broker url without lookups. Meanwhile, during this transfer, any produced messages will be ignored by the source broker. + +Current Unload and Lookup Sequence in Extensible Load Balancer +```mermaid +sequenceDiagram + participant Clients + participant Owner Broker + participant New Owner Broker + participant Leader Broker + 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) + Owner Broker ->> New Owner Broker: "state:Assign:" assign new ownership + New Owner Broker ->> Owner Broker: "state:Owned:" ack new ownership + Clients ->> Owner Broker: lookup + Owner Broker ->> Clients: redirect + Clients ->> New Owner Broker: lookup + New Owner Broker ->> Clients: return(connected) +``` + +Proposed Unload Sequence in Extensible Load Balancer without Lookup +```mermaid +sequenceDiagram + participant Clients + participant Owner Broker + participant New Owner Broker + participant Leader Broker + Leader Broker ->> Owner Broker: "state:Releasing:" close topic + Owner Broker ->> Owner Broker: close broker topic sessions(e.g ledgers) without disconnecting producers/consumers(fenced) + Clients -->> Owner Broker: message pubs are ignored + Owner Broker ->> New Owner Broker: "state:Assign:" assign new ownership + New Owner Broker ->> Owner Broker: "state:Owned:" ack new ownership + Owner Broker ->> Owner Broker: close the fenced broker topic sessions + Owner Broker ->> Clients: close producers and consumers (with newOwnerBrokerUrl) + Clients ->> New Owner Broker: immediately connect +``` + + +# Detailed Design + +## Design & Implementation Details + + + +- Modify CommandCloseProducer, CommandCloseConsumer to pass optional brokerServiceUrls +``` +message CommandCloseProducer { +required uint64 producer_id = 1; +required uint64 request_id = 2; ++ optional string assignedBrokerServiceUrl = 3; ++ optional string assignedBrokerServiceUrlTls = 4; +} + +message CommandCloseConsumer { +required uint64 consumer_id = 1; +required uint64 request_id = 2; ++ optional string assignedBrokerServiceUrl = 3; ++ optional string assignedBrokerServiceUrlTls = 4; +} +``` + +- Add new disconnect apis on producer and consumer to pass dstBrokerLookupData +``` +public CompletableFuture disconnect(Optional dstBrokerLookupData) { +``` + +- Modify the Topic.close() behavior to optionally skip producers.disconnect() and consumers.disconnect(). +``` +public CompletableFuture close(boolean closeWithoutWaitingClientDisconnect, + boolean closeWithoutDisconnectingClients) { + +``` + +- please refer to this poc code for more details: https://github.com/apache/pulsar/compare/master...heesung-sn:pulsar:close-command-dst-url + + +### Eventual Consistency of Ownership States + +This protocol and ownership state checks follow the eventual consistency of the bundle state channel introduced in PIP-192. + +After the client connects to the destination broker, the next command(e.g. ProducerCommand) requires +the destination broker to check the ownership again against its local table view of the bundle state channel. + +Upon this local ownership check, there could be the following scenarios: + +Happy case: +- If the ownership state is `owned ` and the current broker is indeed the owner, the command completes. + +Unhappy cases: +- If the ownership state is `owned ` and the current broker is not the owner, the command fails +(the broker returns an error to the client), and the client tries to find the true new owner by lookups. +The global bundle state channel is eventually consistent, and the lookups should eventually converge. +- if the ownership change is still in progress(`releasing`, `assigning`), this check will be deferred +until the state becomes `owned` with a timeout. + +### Failure Recovery of Ownership States + +The failure recovery logic relies on the bundle state channel cleanup logic introduced in PIP-192. + +When the destination or source broker crashes in the middle of unloading, +the leader will find the orphan state and clean the ownership by selecting a new owner, and the client will reconnect to it. +During this transfer process, if alive, the source broker will serve the topic according to the protocol described in the PIP. + + +## Public-facing Changes + + + +### Public API + + +### Binary protocol + +- Modify CommandCloseProducer, CommandCloseConsumer to pass optional assignedBrokerServiceUrls like the above. + +### Configuration + +### CLI + +### Metrics + + + + +# Monitoring + + + +# Security Considerations + + +# Backward & Forward Compatability +- We are adding new parameters in the close producer and consumer command protocol, the old client versions should not see the optional destination urls in the close commands. Hence, they will request lookups. + +## Revert + + + +## Upgrade + + + +# Alternatives + + + +# General Notes + +# Links + + +* Mailing List discussion thread: +* Mailing List voting thread: + From 1b2a9915c461c4357eb34d14c509963fb6cb47cd Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 18 Oct 2023 09:33:42 +0800 Subject: [PATCH 28/37] [improve][client] Add `REAL_SUBSCRIPTION` when produces msg to DLQ (#21369) --- .../org/apache/pulsar/client/api/DeadLetterTopicTest.java | 4 +++- .../main/java/org/apache/pulsar/client/impl/ConsumerImpl.java | 1 + .../java/org/apache/pulsar/client/util/RetryMessageUtil.java | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) 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 2a0cb3187d208..7be292a602603 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 @@ -234,10 +234,11 @@ public void testDeadLetterTopicHasOriginalInfo() throws Exception { final int maxRedeliveryCount = 1; final int sendMessages = 10; + final String subscriptionName = "my-subscription"; Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) - .subscriptionName("my-subscription") + .subscriptionName(subscriptionName) .subscriptionType(SubscriptionType.Shared) .ackTimeout(1, TimeUnit.SECONDS) .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) @@ -273,6 +274,7 @@ public void testDeadLetterTopicHasOriginalInfo() throws Exception { Message message = deadLetterConsumer.receive(); //Original info should exists assertEquals(message.getProperties().get(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + assertEquals(message.getProperties().get(RetryMessageUtil.SYSTEM_PROPERTY_REAL_SUBSCRIPTION), subscriptionName); assertTrue(messageIds.contains(message.getProperties().get(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID))); deadLetterConsumer.acknowledge(message); totalInDeadLetter++; 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 ded6a546c2403..f390b80a7f01c 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 @@ -719,6 +719,7 @@ private SortedMap getPropertiesMap(Message message, //Compatible with the old version, will be deleted in the future propertiesMap.putIfAbsent(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID, originMessageIdStr); propertiesMap.putIfAbsent(RetryMessageUtil.PROPERTY_ORIGIN_MESSAGE_ID, originMessageIdStr); + propertiesMap.putIfAbsent(RetryMessageUtil.SYSTEM_PROPERTY_REAL_SUBSCRIPTION, subscription); return propertiesMap; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryMessageUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryMessageUtil.java index f73c266877988..e9071f171a29e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryMessageUtil.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryMessageUtil.java @@ -23,6 +23,7 @@ public class RetryMessageUtil { public static final String SYSTEM_PROPERTY_RECONSUMETIMES = "RECONSUMETIMES"; public static final String SYSTEM_PROPERTY_DELAY_TIME = "DELAY_TIME"; public static final String SYSTEM_PROPERTY_REAL_TOPIC = "REAL_TOPIC"; + public static final String SYSTEM_PROPERTY_REAL_SUBSCRIPTION = "REAL_SUBSCRIPTION"; public static final String SYSTEM_PROPERTY_RETRY_TOPIC = "RETRY_TOPIC"; @Deprecated public static final String SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID = "ORIGIN_MESSAGE_IDY_TIME"; From 187e0cf5708b358b844a8b3575ae6e350a39238b Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Wed, 18 Oct 2023 09:57:47 +0800 Subject: [PATCH 29/37] [fix][fn] Make python install dependencies from requirements.txt (#20174) Co-authored-by: tison --- .../instance/src/main/python/python_instance_main.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pulsar-functions/instance/src/main/python/python_instance_main.py b/pulsar-functions/instance/src/main/python/python_instance_main.py index 9a923c7e3a18b..943e1c1c245f5 100755 --- a/pulsar-functions/instance/src/main/python/python_instance_main.py +++ b/pulsar-functions/instance/src/main/python/python_instance_main.py @@ -207,6 +207,15 @@ def main(): zpfile.extractall(os.path.dirname(str(args.py))) basename = os.path.basename(os.path.splitext(str(args.py))[0]) + requirements_file = os.path.join(os.path.dirname(str(args.py)), basename, "requirements.txt") + if os.path.isfile(requirements_file): + cmd = "pip install -r %s" % requirements_file + Log.debug("Install python dependencies via cmd: %s" % cmd) + retval = os.system(cmd) + if retval != 0: + print("Could not install user depedencies specified by the requirements.txt file") + sys.exit(1) + deps_dir = os.path.join(os.path.dirname(str(args.py)), basename, "deps") if os.path.isdir(deps_dir) and os.listdir(deps_dir): From 2ff1b8c6d90e844ce8f8caf3d0c571e86b8fdffb Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Wed, 18 Oct 2023 15:41:01 +0800 Subject: [PATCH 30/37] [improve][pip] PIP-302 Introduce refreshAsync API for TableView (#21271) ## Motivation **Prerequisite:** Since messages are constantly being written into the Topic and there is no read-write lock guarantee, we cannot assure the retrieval of the most up-to-date value. **Implementation Goal:** Record a checkpoint before reading and ensure the retrieval of the latest value of the key up to this checkpoint. **Use Case:** When read and write operations for a certain key do not occur simultaneously, we can refresh the TableView before reading the key to obtain the latest value for this key. --- pip/pip-302.md | 140 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 140 insertions(+) create mode 100644 pip/pip-302.md diff --git a/pip/pip-302.md b/pip/pip-302.md new file mode 100644 index 0000000000000..cf721ee38559b --- /dev/null +++ b/pip/pip-302.md @@ -0,0 +1,140 @@ +# Background Knowledge + +The TableView interface provides a convenient way to access the streaming updatable dataset in a topic by offering a continuously updated key-value map view. The TableView retains the last value of the key which provides you with an almost up-to-date dataset but cannot guarantee you always get the latest data (with the latest written message). + +The TableView can be used to establish a local cache of data. Additionally, clients can register consumers with TableView and specify a listener to scan the map and receive notifications whenever new messages are received. This functionality enables event-driven applications and message monitoring. + +For more detailed information about the TableView, please refer to the [Pulsar documentation](https://pulsar.apache.org/docs/next/concepts-clients/#tableview). + +# Motivation + +When a TableView is created, it retrieves the position of the latest written message and reads all messages from the beginning up to that fetched position. This ensures that the TableView will include any messages written prior to its creation. However, it does not guarantee that the TableView will include any newly added messages during its creation. +Therefore, the value you read from a TableView instance may not be the most recent value, but you will not read an older value once a new value becomes available. It's important to note that this guarantee is not maintained across multiple TableView instances on the same topic. This means that you may receive a newer value from one instance first, and then receive an older value from another instance later. +In addition, we have several other components, such as the transaction buffer snapshot and the topic policies service, that employ a similar mechanism to the TableView. This is because the TableView is not available at that time. However, we cannot replace these implementations with a TableView because they involve multiple TableView instances across brokers within the same system topic, and the data read from these TableViews is not guaranteed to be up-to-date. As a result, subsequent writes may occur based on outdated versions of the data. +For example, in the transaction buffer snapshot, when a broker owns topics within a namespace, it maintains a TableView containing all the transaction buffer snapshots for those topics. It is crucial to ensure that the owner can read the most recently written transaction buffer snapshot when loading a topic (where the topic name serves as the key for the transaction buffer snapshot message). However, the current capabilities provided by TableView do not guarantee this, especially when ownership of the topic is transferred and the TableView of transaction buffer snapshots in the new owner broker is not up-to-date. + +Regarding both the transaction buffer snapshot and topic policies service, updates to a key are only performed by a single writer at a given time until the topic's owner is changed. As a result, it is crucial to ensure that the last written value of this key is read prior to any subsequent writing. By guaranteeing this, all subsequent writes will consistently be based on the most up-to-date value. + +The proposal will introduce a new API to refresh the table view with the latest written data on the topic, ensuring that all subsequent reads are based on the refreshed data. + +```java +tableView.refresh(); +tableView.get(“key”); +``` + +After the refresh, it is ensured that all messages written prior to the refresh will be available to be read. However, it should be noted that the inclusion of newly added messages during or after the refresh is not guaranteed. + +# Goals + +## In Scope + +Providing the capability to refresh the TableView to the last written message of the topic and all the subsequent reads to be conducted using either the refreshed dataset or a dataset that is even more up-to-date than the refreshed one. + +## Out of Scope + + +A static perspective of a TableView at a given moment in time +Read consistency across multiple TableViews on the same topic + +# High-Level Design + +Provide a new API for TableView to support refreshing the dataset of the TableView to the last written message. + +## Design & Implementation Details + +# Public-Facing Changes + +## Public API + +The following changes will be added to the public API of TableView: + +### `refreshAsync()` + +This new API retrieves the position of the latest written message and reads all messages from the beginning up to that fetched position. This ensures that the TableView will include any messages written prior to its refresh. + +```java +/** +* +* Refresh the table view with the latest data in the topic, ensuring that all subsequent reads are based on the refreshed data. +* +* Example usage: +* +* table.refreshAsync().thenApply(__ -> table.get(key)); +* +* This function retrieves the last written message in the topic and refreshes the table view accordingly. +* Once the refresh is complete, all subsequent reads will be performed on the refreshed data or a combination of the refreshed +* data and newly published data. The table view remains synchronized with any newly published data after the refresh. +* +* |x:0|->|y:0|->|z:0|->|x:1|->|z:1|->|x:2|->|y:1|->|y:2| +* +* If a read occurs after the refresh (at the last published message |y:2|), it ensures that outdated data like x=1 is not obtained. +* However, it does not guarantee that the values will always be x=2, y=2, z=1, as the table view may receive updates with newly +* published data. +* +* |x:0|->|y:0|->|z:0|->|x:1|->|z:1|->|x:2|->|y:1|->|y:2| -> |y:3| +* +* Both y=2 or y=3 are possible. Therefore, different readers may receive different values, but all values will be equal to or newer +* than the data refreshed from the last call to the refresh method. +*/ +CompletableFuture refreshAsync(); + +/** +* Refresh the table view with the latest data in the topic, ensuring that all subsequent reads are based on the refreshed data. +* +* @throws PulsarClientException if there is any error refreshing the table view. +*/ +void refresh() throws PulsarClientException; + + +``` + +# Monitoring + +The proposed changes do not introduce any specific monitoring considerations at this time. + +# Security Considerations + +No specific security considerations have been identified for this proposal. + +# Backward & Forward Compatibility + +## Revert + +No specific revert instructions are required for this proposal. + +## Upgrade + +No specific upgrade instructions are required for this proposal. + +# Alternatives + +## Add consistency model policy to TableView +Add new option configuration `STRONG_CONSISTENCY_MODEL` and `EVENTUAL_CONSISTENCY_MODEL` in TableViewConfigurationData. +• `STRONG_CONSISTENCY_MODEL`: any method will be blocked until the latest value is retrieved. +• `EVENTUAL_CONSISTENCY_MODEL`: all methods are non-blocking, but the value retrieved might not be the latest at the time point. + +However, there might be some drawbacks to this approach: +1. As read and write operations might happen simultaneously, we cannot guarantee consistency. If we provide a configuration about consistency, it might confuse users. +2. This operation will block each get operation. We need to add more asynchronous methods. +3. Less flexibility if users don’t want to refresh the TableView for any reads. + +## New method for combining the refresh and get + +Another option is to add new methods for the existing methods to combine the refresh and reads. For example + +CompletableFuture refreshGet(String key); + +It will refresh the dataset of the TableView and perform the get operation based on the refreshed dataset. But we need to add 11 new methods to the public APIs of the TableView. + + +# General Notes + +No additional general notes have been provided. + +# Links + + +* Mailing List discussion thread: +* Mailing List voting thread: From 03915ebbdbf1cf8d300e98778cd8cf77ea870d23 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 18 Oct 2023 18:38:59 +0800 Subject: [PATCH 31/37] [fix][doc] Fix typos in doc for Validator class (#21323) --- .../java/org/apache/pulsar/config/validation/Validator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-config-validation/src/main/java/org/apache/pulsar/config/validation/Validator.java b/pulsar-config-validation/src/main/java/org/apache/pulsar/config/validation/Validator.java index ea3332d886c9e..d850b2b654568 100644 --- a/pulsar-config-validation/src/main/java/org/apache/pulsar/config/validation/Validator.java +++ b/pulsar-config-validation/src/main/java/org/apache/pulsar/config/validation/Validator.java @@ -32,7 +32,7 @@ public Validator() { } /** - * validate the field value o that belogs to the field which is named name + * validate the field value o that belongs to the field which is named name * This method should throw IllegalArgumentException in case o doesn't * validate per this validator's implementation. */ From 9b643c899cbc99b2ead131aed45388c99615a8e7 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 18 Oct 2023 18:39:26 +0800 Subject: [PATCH 32/37] [fix][doc] Fix typos in doc for broker.conf (#21322) --- conf/broker.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index ca407810a42fc..a043f379ed478 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -445,7 +445,7 @@ dispatcherReadFailureBackoffMaxTimeInMs=60000 # The read failure backoff mandatory stop time in milliseconds. By default it is 0s. dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 -# 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 # Class name of Pluggable entry filter that can decide whether the entry needs to be filtered From b1bca5609d254734ccca63b616eba33ce3a8b70b Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Wed, 18 Oct 2023 18:56:47 +0800 Subject: [PATCH 33/37] [improve][broker][PIP-286] Make the TopicCompactionService to support find entry based on publishTime or index (#21208) --- .../PulsarTopicCompactionService.java | 75 +++++++++++++++++++ .../compaction/TopicCompactionService.java | 17 +++++ .../TopicCompactionServiceTest.java | 55 ++++++++++++-- 3 files changed, 142 insertions(+), 5 deletions(-) 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 c420767d1e884..1d3f94dcb9048 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 @@ -22,18 +22,23 @@ import static org.apache.pulsar.compaction.CompactedTopicImpl.COMPACT_LEDGER_EMPTY; import static org.apache.pulsar.compaction.CompactedTopicImpl.NEWER_THAN_COMPACTED; import static org.apache.pulsar.compaction.CompactedTopicImpl.findStartPoint; +import static org.apache.pulsar.compaction.CompactedTopicImpl.readEntries; import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.function.Predicate; import java.util.function.Supplier; import javax.annotation.Nonnull; import org.apache.bookkeeper.client.BookKeeper; +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.pulsar.common.api.proto.BrokerEntryMetadata; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; @@ -106,6 +111,76 @@ public CompletableFuture getLastCompactedPosition() { return CompletableFuture.completedFuture(compactedTopic.getCompactionHorizon().orElse(null)); } + @Override + public CompletableFuture findEntryByPublishTime(long publishTime) { + final Predicate predicate = entry -> { + return Commands.parseMessageMetadata(entry.getDataBuffer()).getPublishTime() >= publishTime; + }; + return findFirstMatchEntry(predicate); + } + + @Override + public CompletableFuture findEntryByEntryIndex(long entryIndex) { + final Predicate predicate = entry -> { + BrokerEntryMetadata brokerEntryMetadata = Commands.parseBrokerEntryMetadataIfExist(entry.getDataBuffer()); + if (brokerEntryMetadata == null || !brokerEntryMetadata.hasIndex()) { + return false; + } + return brokerEntryMetadata.getIndex() >= entryIndex; + }; + return findFirstMatchEntry(predicate); + } + + private CompletableFuture findFirstMatchEntry(final Predicate predicate) { + var compactedTopicContextFuture = compactedTopic.getCompactedTopicContextFuture(); + + if (compactedTopicContextFuture == null) { + return CompletableFuture.completedFuture(null); + } + return compactedTopicContextFuture.thenCompose(compactedTopicContext -> { + LedgerHandle lh = compactedTopicContext.getLedger(); + CompletableFuture promise = new CompletableFuture<>(); + findFirstMatchIndexLoop(predicate, 0L, lh.getLastAddConfirmed(), promise, null, lh); + return promise.thenCompose(index -> { + if (index == null) { + return CompletableFuture.completedFuture(null); + } + return readEntries(lh, index, index).thenApply(entries -> entries.get(0)); + }); + }); + } + + private static void findFirstMatchIndexLoop(final Predicate predicate, + final long start, final long end, + final CompletableFuture promise, + final Long lastMatchIndex, + final LedgerHandle lh) { + if (start > end) { + promise.complete(lastMatchIndex); + return; + } + + long mid = (start + end) / 2; + readEntries(lh, mid, mid).thenAccept(entries -> { + Entry entry = entries.get(0); + final boolean isMatch; + try { + isMatch = predicate.test(entry); + } finally { + entry.release(); + } + + if (isMatch) { + findFirstMatchIndexLoop(predicate, start, mid - 1, promise, mid, lh); + } else { + findFirstMatchIndexLoop(predicate, mid + 1, end, promise, lastMatchIndex, lh); + } + }).exceptionally(ex -> { + promise.completeExceptionally(ex); + return null; + }); + } + public CompactedTopicImpl getCompactedTopic() { return compactedTopic; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TopicCompactionService.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TopicCompactionService.java index 74df0dafabdcd..fdd6bebbdec33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TopicCompactionService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TopicCompactionService.java @@ -60,4 +60,21 @@ public interface TopicCompactionService extends AutoCloseable { * @return a future that will be completed with the last compacted position, this position can be null. */ CompletableFuture getLastCompactedPosition(); + + /** + * Find the first entry that greater or equal to target publishTime. + * + * @param publishTime the publish time of entry. + * @return the first entry metadata that greater or equal to target publishTime, this entry can be null. + */ + CompletableFuture findEntryByPublishTime(long publishTime); + + /** + * Find the first entry that greater or equal to target entryIndex, + * if an entry that broker entry metadata is missed, then it will be skipped and find the next match entry. + * + * @param entryIndex the index of entry. + * @return the first entry that greater or equal to target entryIndex, this entry can be null. + */ + CompletableFuture findEntryByEntryIndex(long entryIndex); } 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 4abe00fb0c631..d84d1ccc9ea45 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 @@ -21,6 +21,8 @@ import static org.apache.pulsar.compaction.Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import static org.testng.Assert.assertEquals; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.fail; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.util.List; @@ -35,13 +37,14 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; -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.PulsarClientException; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; +import org.apache.pulsar.common.api.proto.MessageMetadata; 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.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -55,6 +58,8 @@ public class TopicCompactionServiceTest extends MockedPulsarServiceBaseTest { @BeforeMethod @Override public void setup() throws Exception { + conf.setExposingBrokerEntryMetadataToClientEnabled(true); + super.internalSetup(); admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); @@ -82,7 +87,7 @@ public void cleanup() throws Exception { } @Test - public void test() throws PulsarClientException, PulsarAdminException { + public void test() throws Exception { String topic = "persistent://prop-xyz/ns1/my-topic"; PulsarTopicCompactionService service = new PulsarTopicCompactionService(topic, bk, () -> compactor); @@ -93,6 +98,18 @@ public void test() throws PulsarClientException, PulsarAdminException { .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); + producer.newMessage() + .key("c") + .value("C_0".getBytes()) + .send(); + + conf.setBrokerEntryMetadataInterceptors(org.assertj.core.util.Sets.newTreeSet( + "org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor" + )); + restartBroker(); + + long startTime = System.currentTimeMillis(); + producer.newMessage() .key("a") .value("A_1".getBytes()) @@ -133,7 +150,7 @@ public void test() throws PulsarClientException, PulsarAdminException { assertEquals(admin.topics().getInternalStats(topic).lastConfirmedEntry, lastCompactedPosition.toString()); List entries = service.readCompactedEntries(PositionImpl.EARLIEST, 4).join(); - assertEquals(entries.size(), 2); + assertEquals(entries.size(), 3); entries.stream().map(e -> { try { return MessageImpl.deserialize(e.getDataBuffer()); @@ -144,12 +161,40 @@ public void test() throws PulsarClientException, PulsarAdminException { String data = new String(message.getData()); if (Objects.equals(message.getKey(), "a")) { assertEquals(data, "A_2"); - } else { + } else if (Objects.equals(message.getKey(), "b")) { assertEquals(data, "B_3"); + } else if (Objects.equals(message.getKey(), "c")) { + assertEquals(data, "C_0"); + } else { + fail(); } }); List entries2 = service.readCompactedEntries(PositionImpl.EARLIEST, 1).join(); assertEquals(entries2.size(), 1); + + Entry entry = service.findEntryByEntryIndex(0).join(); + BrokerEntryMetadata brokerEntryMetadata = Commands.peekBrokerEntryMetadataIfExist(entry.getDataBuffer()); + assertNotNull(brokerEntryMetadata); + assertEquals(brokerEntryMetadata.getIndex(), 2); + MessageMetadata metadata = Commands.parseMessageMetadata(entry.getDataBuffer()); + assertEquals(metadata.getPartitionKey(), "a"); + entry.release(); + + entry = service.findEntryByEntryIndex(3).join(); + brokerEntryMetadata = Commands.peekBrokerEntryMetadataIfExist(entry.getDataBuffer()); + assertNotNull(brokerEntryMetadata); + assertEquals(brokerEntryMetadata.getIndex(), 4); + metadata = Commands.parseMessageMetadata(entry.getDataBuffer()); + assertEquals(metadata.getPartitionKey(), "b"); + entry.release(); + + entry = service.findEntryByPublishTime(startTime).join(); + brokerEntryMetadata = Commands.peekBrokerEntryMetadataIfExist(entry.getDataBuffer()); + assertNotNull(brokerEntryMetadata); + assertEquals(brokerEntryMetadata.getIndex(), 2); + metadata = Commands.parseMessageMetadata(entry.getDataBuffer()); + assertEquals(metadata.getPartitionKey(), "a"); + entry.release(); } } From a9d5d25e52710503cc2642ba7a8aadda0a32faae Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 15:28:57 +0300 Subject: [PATCH 34/37] [improve][build] Upgrade Jacoco version to 0.8.11 to support Java 21 (#21388) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2ac82aaee7618..63c1350df5e64 100644 --- a/pom.xml +++ b/pom.xml @@ -288,7 +288,7 @@ flexible messaging model and an intuitive client API. 4.9.10 3.5.3 1.7.0 - 0.8.8 + 0.8.11 4.7.3.0 4.7.3 2.5.1 From e2c6c08b3598e01aba1bac90b56412e451140385 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 18:21:03 +0300 Subject: [PATCH 35/37] [improve][ci] Add new CI unit test group "Broker Group 4" with cluster migration tests (#21391) --- .github/workflows/pulsar-ci.yaml | 2 ++ build/run_unit_group.sh | 4 ++++ .../apache/pulsar/broker/service/ClusterMigrationTest.java | 2 +- 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index e067e42f43c8d..6ada207b786d2 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -189,6 +189,8 @@ jobs: group: BROKER_GROUP_2 - name: Brokers - Broker Group 3 group: BROKER_GROUP_3 + - name: Brokers - Broker Group 4 + group: BROKER_GROUP_4 - name: Brokers - Client Api group: BROKER_CLIENT_API - name: Brokers - Client Impl diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 69434b011b37e..17d0efeed9937 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -87,6 +87,10 @@ function test_group_broker_group_3() { mvn_test -pl pulsar-broker -Dgroups='broker-admin' } +function test_group_broker_group_4() { + mvn_test -pl pulsar-broker -Dgroups='cluster-migration' +} + function test_group_broker_client_api() { mvn_test -pl pulsar-broker -Dgroups='broker-api' } 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 aa5444edc9901..2139a7bc12ed2 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 @@ -55,7 +55,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -@Test(groups = "broker") +@Test(groups = "cluster-migration") public class ClusterMigrationTest { private static final Logger log = LoggerFactory.getLogger(ClusterMigrationTest.class); From 5af821d3743321d7546637fc3595801fbda791e9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 18:21:18 +0300 Subject: [PATCH 36/37] [improve][build] Upgrade spotbugs maven plugin version for Java 21 compatibility (#21389) --- pom.xml | 2 +- .../src/main/resources/findbugsExclude.xml | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 63c1350df5e64..70ccc6050b5cf 100644 --- a/pom.xml +++ b/pom.xml @@ -289,7 +289,7 @@ flexible messaging model and an intuitive client API. 3.5.3 1.7.0 0.8.11 - 4.7.3.0 + 4.7.3.6 4.7.3 2.5.1 9+181-r4173-1 diff --git a/pulsar-client/src/main/resources/findbugsExclude.xml b/pulsar-client/src/main/resources/findbugsExclude.xml index 67c012ad30b8f..f47f9b4a31a09 100644 --- a/pulsar-client/src/main/resources/findbugsExclude.xml +++ b/pulsar-client/src/main/resources/findbugsExclude.xml @@ -337,6 +337,11 @@ + + + + + @@ -387,6 +392,11 @@ + + + + + @@ -427,6 +437,11 @@ + + + + + @@ -447,6 +462,11 @@ + + + + + From c8a2f49c6c6edaf6b5667f9ac7df65b815aefe58 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Thu, 19 Oct 2023 10:18:12 +0800 Subject: [PATCH 37/37] [fix][broker] Fix heartbeat namespace create transaction internal topic (#21348) --- .../service/persistent/PersistentTopic.java | 3 ++- .../systopic/PartitionedSystemTopicTest.java | 19 +++++++++++++++++++ 2 files changed, 21 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 03ee0f06e2fbb..4fe555aca5f27 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 @@ -319,7 +319,8 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS TopicName topicName = TopicName.get(topic); if (brokerService.getPulsar().getConfiguration().isTransactionCoordinatorEnabled() - && !isEventSystemTopic(topicName)) { + && !isEventSystemTopic(topicName) + && !NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { this.transactionBuffer = brokerService.getPulsar() .getTransactionBufferProvider().newTransactionBuffer(this); } else { 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 4af0bd9052391..42d941e616809 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 @@ -81,6 +81,7 @@ protected void setup() throws Exception { conf.setDefaultNumPartitions(PARTITIONS); conf.setManagedLedgerMaxEntriesPerLedger(1); conf.setBrokerDeleteInactiveTopicsEnabled(false); + conf.setTransactionCoordinatorEnabled(true); super.baseSetup(); } @@ -207,6 +208,24 @@ public void testHeartbeatTopicNotAllowedToSendEvent() throws Exception { }); } + @Test + public void testHeartbeatNamespaceNotCreateTransactionInternalTopic() throws Exception { + admin.brokers().healthcheck(TopicVersion.V2); + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), + pulsar.getConfig()); + TopicName topicName = TopicName.get("persistent", + namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); + Optional optionalTopic = pulsar.getBrokerService() + .getTopic(topicName.getPartition(1).toString(), false).join(); + Assert.assertTrue(optionalTopic.isEmpty()); + + List topics = getPulsar().getNamespaceService().getListOfPersistentTopics(namespaceName).join(); + Assert.assertEquals(topics.size(), 1); + TopicName heartbeatTopicName = TopicName.get("persistent", + namespaceName, BrokersBase.HEALTH_CHECK_TOPIC_SUFFIX); + Assert.assertEquals(topics.get(0), heartbeatTopicName.toString()); + } + @Test public void testSetBacklogCausedCreatingProducerFailure() throws Exception { final String ns = "prop/ns-test";