From b83dab5f7671cb47ca3dfa18938c0d10b9ba1de3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 4 Mar 2024 18:12:08 +0200 Subject: [PATCH 01/16] [improve][fn] Add configuration for connector & functions package url sources (#22184) (cherry picked from commit 207335a449f2bc9cdf6782c67f93f8c2fb267271) (cherry picked from commit b107387a8296f4f2fca310953dacee722e485709) --- conf/functions_worker.yml | 8 + .../worker/PulsarFunctionE2ESecurityTest.java | 8 +- .../worker/PulsarFunctionPublishTest.java | 6 +- .../worker/PulsarFunctionTlsTest.java | 16 +- .../pulsar/io/AbstractPulsarE2ETest.java | 11 +- .../pulsar/functions/worker/WorkerConfig.java | 24 +++ .../functions/worker/FunctionActioner.java | 174 +++++++++++------- .../worker/FunctionRuntimeManager.java | 3 +- .../functions/worker/PackageUrlValidator.java | 102 ++++++++++ .../functions/worker/PulsarWorkerService.java | 4 +- .../worker/rest/api/ComponentImpl.java | 28 ++- .../worker/rest/api/FunctionsImpl.java | 83 ++------- .../functions/worker/rest/api/SinksImpl.java | 3 +- .../worker/rest/api/SourcesImpl.java | 3 +- .../worker/FunctionActionerTest.java | 93 +++++----- .../worker/FunctionRuntimeManagerTest.java | 2 +- .../v3/AbstractFunctionApiResourceTest.java | 1 - .../api/v3/AbstractFunctionsResourceTest.java | 18 +- .../rest/api/v3/SinkApiV3ResourceTest.java | 48 +++-- 19 files changed, 396 insertions(+), 239 deletions(-) create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 20a2f25e3b2ad..5310044a97466 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -398,7 +398,15 @@ saslJaasServerRoleTokenSignerSecretPath: ######################## connectorsDirectory: ./connectors +# Whether to enable referencing connectors directory files by file url in connector (sink/source) creation +enableReferencingConnectorDirectoryFiles: true +# Regex patterns for enabling creation of connectors by referencing packages in matching http/https urls +additionalEnabledConnectorUrlPatterns: [] functionsDirectory: ./functions +# Whether to enable referencing functions directory files by file url in functions creation +enableReferencingFunctionsDirectoryFiles: true +# Regex patterns for enabling creation of functions by referencing packages in matching http/https urls +additionalEnabledFunctionsUrlPatterns: [] # Enables extended validation for connector config with fine-grain annotation based validation # during submission. Classloading with either enableClassloadingOfExternalFiles or 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 173794a2cfb58..921ee8f5d6443 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 @@ -27,14 +27,15 @@ import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.google.common.collect.Lists; import com.google.common.collect.Sets; import io.jsonwebtoken.SignatureAlgorithm; import java.lang.reflect.Method; import java.net.URL; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; @@ -265,6 +266,11 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationEnabled(config.isAuthorizationEnabled()); workerConfig.setAuthorizationProvider(config.getAuthorizationProvider()); + List urlPatterns = + Arrays.asList(getPulsarApiExamplesJar().getParentFile().toURI() + ".*", "http://127\\.0\\.0\\.1:.*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + 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 d985241e2903d..7df84459322ed 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 @@ -25,13 +25,13 @@ import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; - import com.google.common.collect.Lists; import com.google.common.collect.Sets; import java.io.File; import java.lang.reflect.Method; import java.net.URI; import java.net.URL; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -260,6 +260,10 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); + List urlPatterns = Arrays.asList(getPulsarApiExamplesJar().getParentFile().toURI() + ".*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + PulsarWorkerService workerService = new PulsarWorkerService(); workerService.init(workerConfig, null, false); return workerService; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java index f4a27506c2e10..28b0b2856786d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java @@ -26,9 +26,11 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -71,6 +73,7 @@ public class PulsarFunctionTlsTest { protected PulsarService[] pulsarServices = new PulsarService[BROKER_COUNT]; protected PulsarService leaderPulsar; protected PulsarAdmin leaderAdmin; + protected WorkerService[] fnWorkerServices = new WorkerService[BROKER_COUNT]; protected String testCluster = "my-cluster"; protected String testTenant = "my-tenant"; protected String testNamespace = testTenant + "/my-ns"; @@ -136,12 +139,18 @@ void setup() throws Exception { workerConfig.setBrokerClientAuthenticationEnabled(true); workerConfig.setTlsEnabled(true); workerConfig.setUseTls(true); - WorkerService fnWorkerService = WorkerServiceLoader.load(workerConfig); + File packagePath = new File( + PulsarSink.class.getProtectionDomain().getCodeSource().getLocation().getPath()).getParentFile(); + List urlPatterns = + Arrays.asList(packagePath.toURI() + ".*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + fnWorkerServices[i] = WorkerServiceLoader.load(workerConfig); configurations[i] = config; pulsarServices[i] = new PulsarService( - config, workerConfig, Optional.of(fnWorkerService), code -> {}); + config, workerConfig, Optional.of(fnWorkerServices[i]), code -> {}); pulsarServices[i].start(); // Sleep until pulsarServices[0] becomes leader, this way we can spy namespace bundle assignment easily. @@ -180,6 +189,9 @@ void tearDown() throws Exception { if (pulsarAdmins[i] != null) { pulsarAdmins[i].close(); } + if (fnWorkerServices[i] != null) { + fnWorkerServices[i].stop(); + } if (pulsarServices[i] != null) { pulsarServices[i].close(); } 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 9e1edea2f8029..96521094b9f71 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 @@ -25,7 +25,7 @@ import static org.apache.pulsar.functions.worker.PulsarFunctionLocalRunTest.getPulsarIODataGeneratorNar; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertTrue; - +import com.google.common.collect.Sets; import java.io.File; import java.io.IOException; import java.lang.reflect.Method; @@ -34,10 +34,10 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; - import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.ServiceConfigurationUtils; @@ -70,8 +70,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.Sets; - public abstract class AbstractPulsarE2ETest { public static final Logger log = LoggerFactory.getLogger(AbstractPulsarE2ETest.class); @@ -288,6 +286,11 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); + List urlPatterns = + Arrays.asList(getPulsarApiExamplesJar().getParentFile().toURI() + ".*", "http://127\\.0\\.0\\.1:.*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + PulsarWorkerService workerService = new PulsarWorkerService(); workerService.init(workerConfig, null, false); return workerService; diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index cd4c6c3da7fc1..9be6272aa3746 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -237,6 +237,18 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { doc = "The directory where nar packages are extractors" ) private String narExtractionDirectory = NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR; + @FieldContext( + category = CATEGORY_CONNECTORS, + doc = "Whether to enable referencing connectors directory files by file url in connector (sink/source) " + + "creation. Default is true." + ) + private Boolean enableReferencingConnectorDirectoryFiles = true; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "Regex patterns for enabling creation of connectors by referencing packages in matching http/https " + + "urls." + ) + private List additionalEnabledConnectorUrlPatterns = new ArrayList<>(); @FieldContext( category = CATEGORY_CONNECTORS, doc = "Enables extended validation for connector config with fine-grain annotation based validation " @@ -255,6 +267,18 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { doc = "The path to the location to locate builtin functions" ) private String functionsDirectory = "./functions"; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "Whether to enable referencing functions directory files by file url in functions creation. " + + "Default is true." + ) + private Boolean enableReferencingFunctionsDirectoryFiles = true; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "Regex patterns for enabling creation of functions by referencing packages in matching http/https " + + "urls." + ) + private List additionalEnabledFunctionsUrlPatterns = new ArrayList<>(); @FieldContext( category = CATEGORY_FUNC_METADATA_MNG, doc = "The Pulsar topic used for storing function metadata" diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java index a982604b05627..a592a1726a7bb 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java @@ -29,7 +29,22 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.io.MoreFiles; import com.google.common.io.RecursiveDeleteOption; - +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.stream.Collectors; import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -57,23 +72,6 @@ import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.net.URL; -import java.nio.file.FileAlreadyExistsException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.UUID; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; - @Data @Slf4j public class FunctionActioner { @@ -84,17 +82,21 @@ public class FunctionActioner { private final ConnectorsManager connectorsManager; private final FunctionsManager functionsManager; private final PulsarAdmin pulsarAdmin; + private final PackageUrlValidator packageUrlValidator; public FunctionActioner(WorkerConfig workerConfig, RuntimeFactory runtimeFactory, Namespace dlogNamespace, - ConnectorsManager connectorsManager,FunctionsManager functionsManager,PulsarAdmin pulsarAdmin) { + ConnectorsManager connectorsManager, + FunctionsManager functionsManager, PulsarAdmin pulsarAdmin, + PackageUrlValidator packageUrlValidator) { this.workerConfig = workerConfig; this.runtimeFactory = runtimeFactory; this.dlogNamespace = dlogNamespace; this.connectorsManager = connectorsManager; this.functionsManager = functionsManager; this.pulsarAdmin = pulsarAdmin; + this.packageUrlValidator = packageUrlValidator; } @@ -109,29 +111,13 @@ public void startFunction(FunctionRuntimeInfo functionRuntimeInfo) { String packageFile; - String pkgLocation = functionMetaData.getPackageLocation().getPackagePath(); - boolean isPkgUrlProvided = isFunctionPackageUrlSupported(pkgLocation); + Function.PackageLocationMetaData pkgLocation = functionMetaData.getPackageLocation(); if (runtimeFactory.externallyManaged()) { - packageFile = pkgLocation; + packageFile = pkgLocation.getPackagePath(); } else { - if (isPkgUrlProvided && pkgLocation.startsWith(FILE)) { - URL url = new URL(pkgLocation); - File pkgFile = new File(url.toURI()); - packageFile = pkgFile.getAbsolutePath(); - } else if (FunctionCommon.isFunctionCodeBuiltin(functionDetails)) { - File pkgFile = getBuiltinArchive(FunctionDetails.newBuilder(functionMetaData.getFunctionDetails())); - packageFile = pkgFile.getAbsolutePath(); - } else { - File pkgDir = new File(workerConfig.getDownloadDirectory(), - getDownloadPackagePath(functionMetaData, instanceId)); - pkgDir.mkdirs(); - File pkgFile = new File( - pkgDir, - new File(getDownloadFileName(functionMetaData.getFunctionDetails(), functionMetaData.getPackageLocation())).getName()); - downloadFile(pkgFile, isPkgUrlProvided, functionMetaData, instanceId); - packageFile = pkgFile.getAbsolutePath(); - } + packageFile = getPackageFile(functionMetaData, functionDetails, instanceId, pkgLocation, + InstanceUtils.calculateSubjectType(functionDetails)); } // Setup for batch sources if necessary @@ -150,11 +136,45 @@ public void startFunction(FunctionRuntimeInfo functionRuntimeInfo) { } } + private String getPackageFile(FunctionMetaData functionMetaData, FunctionDetails functionDetails, int instanceId, + Function.PackageLocationMetaData pkgLocation, + FunctionDetails.ComponentType componentType) + throws URISyntaxException, IOException, ClassNotFoundException, PulsarAdminException { + String packagePath = pkgLocation.getPackagePath(); + boolean isPkgUrlProvided = isFunctionPackageUrlSupported(packagePath); + String packageFile; + if (isPkgUrlProvided && packagePath.startsWith(FILE)) { + if (!packageUrlValidator.isValidPackageUrl(componentType, packagePath)) { + throw new IllegalArgumentException("Package URL " + packagePath + " is not valid"); + } + URL url = new URL(packagePath); + File pkgFile = new File(url.toURI()); + packageFile = pkgFile.getAbsolutePath(); + } else if (FunctionCommon.isFunctionCodeBuiltin(functionDetails, componentType)) { + File pkgFile = getBuiltinArchive( + componentType, + FunctionDetails.newBuilder(functionMetaData.getFunctionDetails())); + packageFile = pkgFile.getAbsolutePath(); + } else { + File pkgDir = new File(workerConfig.getDownloadDirectory(), + getDownloadPackagePath(functionMetaData, instanceId)); + pkgDir.mkdirs(); + File pkgFile = new File( + pkgDir, + new File(getDownloadFileName(functionMetaData.getFunctionDetails(), + pkgLocation)).getName()); + downloadFile(pkgFile, isPkgUrlProvided, functionMetaData, instanceId, pkgLocation, componentType); + packageFile = pkgFile.getAbsolutePath(); + } + return packageFile; + } + RuntimeSpawner getRuntimeSpawner(Function.Instance instance, String packageFile) { FunctionMetaData functionMetaData = instance.getFunctionMetaData(); int instanceId = instance.getInstanceId(); - FunctionDetails.Builder functionDetailsBuilder = FunctionDetails.newBuilder(functionMetaData.getFunctionDetails()); + FunctionDetails.Builder functionDetailsBuilder = FunctionDetails + .newBuilder(functionMetaData.getFunctionDetails()); // check to make sure functionAuthenticationSpec has any data and authentication is enabled. // If not set to null, since for protobuf, @@ -197,14 +217,15 @@ InstanceConfig createInstanceConfig(FunctionDetails functionDetails, Function.Fu } private void downloadFile(File pkgFile, boolean isPkgUrlProvided, FunctionMetaData functionMetaData, - int instanceId) throws FileNotFoundException, IOException, PulsarAdminException { + int instanceId, Function.PackageLocationMetaData pkgLocation, + FunctionDetails.ComponentType componentType) + throws IOException, PulsarAdminException { FunctionDetails details = functionMetaData.getFunctionDetails(); File pkgDir = pkgFile.getParentFile(); if (pkgFile.exists()) { - log.warn("Function package exists already {} deleting it", - pkgFile); + log.warn("Function package exists already {} deleting it", pkgFile); pkgFile.delete(); } @@ -212,16 +233,19 @@ private void downloadFile(File pkgFile, boolean isPkgUrlProvided, FunctionMetaDa do { tempPkgFile = new File( pkgDir, - pkgFile.getName() + "." + instanceId + "." + UUID.randomUUID().toString()); + pkgFile.getName() + "." + instanceId + "." + UUID.randomUUID()); } while (tempPkgFile.exists() || !tempPkgFile.createNewFile()); - String pkgLocationPath = functionMetaData.getPackageLocation().getPackagePath(); + String pkgLocationPath = pkgLocation.getPackagePath(); boolean downloadFromHttp = isPkgUrlProvided && pkgLocationPath.startsWith(HTTP); boolean downloadFromPackageManagementService = isPkgUrlProvided && hasPackageTypePrefix(pkgLocationPath); log.info("{}/{}/{} Function package file {} will be downloaded from {}", tempPkgFile, details.getTenant(), details.getNamespace(), details.getName(), - downloadFromHttp ? pkgLocationPath : functionMetaData.getPackageLocation()); + downloadFromHttp ? pkgLocationPath : pkgLocation); - if(downloadFromHttp) { + if (downloadFromHttp) { + if (!packageUrlValidator.isValidPackageUrl(componentType, pkgLocationPath)) { + throw new IllegalArgumentException("Package URL " + pkgLocationPath + " is not valid"); + } FunctionCommon.downloadFromHttpUrl(pkgLocationPath, tempPkgFile); } else if (downloadFromPackageManagementService) { getPulsarAdmin().packages().download(pkgLocationPath, tempPkgFile.getPath()); @@ -248,13 +272,13 @@ private void downloadFile(File pkgFile, boolean isPkgUrlProvided, FunctionMetaDa } catch (FileAlreadyExistsException faee) { // file already exists log.warn("Function package has been downloaded from {} and saved at {}", - functionMetaData.getPackageLocation(), pkgFile); + pkgLocation, pkgFile); } } finally { tempPkgFile.delete(); } - if(details.getRuntime() == Function.FunctionDetails.Runtime.GO && !pkgFile.canExecute()) { + if (details.getRuntime() == Function.FunctionDetails.Runtime.GO && !pkgFile.canExecute()) { pkgFile.setExecutable(true); log.info("Golang function package file {} is set to executable", pkgFile); } @@ -296,7 +320,7 @@ public void stopFunction(FunctionRuntimeInfo functionRuntimeInfo) { public void terminateFunction(FunctionRuntimeInfo functionRuntimeInfo) { FunctionDetails details = functionRuntimeInfo.getFunctionInstance().getFunctionMetaData().getFunctionDetails(); String fqfn = FunctionCommon.getFullyQualifiedName(details); - log.info("{}-{} Terminating function...", fqfn,functionRuntimeInfo.getFunctionInstance().getInstanceId()); + log.info("{}-{} Terminating function...", fqfn, functionRuntimeInfo.getFunctionInstance().getInstanceId()); if (functionRuntimeInfo.getRuntimeSpawner() != null) { functionRuntimeInfo.getRuntimeSpawner().close(); @@ -306,13 +330,15 @@ public void terminateFunction(FunctionRuntimeInfo functionRuntimeInfo) { functionRuntimeInfo.getRuntimeSpawner() .getRuntimeFactory().getAuthProvider().ifPresent(functionAuthProvider -> { try { - log.info("{}-{} Cleaning up authentication data for function...", fqfn,functionRuntimeInfo.getFunctionInstance().getInstanceId()); + log.info("{}-{} Cleaning up authentication data for function...", fqfn, + functionRuntimeInfo.getFunctionInstance().getInstanceId()); functionAuthProvider .cleanUpAuthData( details, Optional.ofNullable(getFunctionAuthData( Optional.ofNullable( - functionRuntimeInfo.getRuntimeSpawner().getInstanceConfig().getFunctionAuthenticationSpec())))); + functionRuntimeInfo.getRuntimeSpawner().getInstanceConfig() + .getFunctionAuthenticationSpec())))); } catch (Exception e) { log.error("Failed to cleanup auth data for function: {}", fqfn, e); @@ -334,11 +360,15 @@ public void accept(Map.Entry stringConsumerSpecEn Function.ConsumerSpec consumerSpec = stringConsumerSpecEntry.getValue(); String topic = stringConsumerSpecEntry.getKey(); - String subscriptionName = isBlank(functionRuntimeInfo.getFunctionInstance().getFunctionMetaData().getFunctionDetails().getSource().getSubscriptionName()) - ? InstanceUtils.getDefaultSubscriptionName(functionRuntimeInfo.getFunctionInstance().getFunctionMetaData().getFunctionDetails()) - : functionRuntimeInfo.getFunctionInstance().getFunctionMetaData().getFunctionDetails().getSource().getSubscriptionName(); + String subscriptionName = isBlank(functionRuntimeInfo.getFunctionInstance() + .getFunctionMetaData().getFunctionDetails().getSource().getSubscriptionName()) + ? InstanceUtils.getDefaultSubscriptionName(functionRuntimeInfo + .getFunctionInstance().getFunctionMetaData().getFunctionDetails()) + : functionRuntimeInfo.getFunctionInstance().getFunctionMetaData() + .getFunctionDetails().getSource().getSubscriptionName(); - deleteSubscription(topic, consumerSpec, subscriptionName, String.format("Cleaning up subscriptions for function %s", fqfn)); + deleteSubscription(topic, consumerSpec, subscriptionName, + String.format("Cleaning up subscriptions for function %s", fqfn)); } }); } @@ -347,7 +377,8 @@ public void accept(Map.Entry stringConsumerSpecEn cleanupBatchSource(details); } - private void deleteSubscription(String topic, Function.ConsumerSpec consumerSpec, String subscriptionName, String msg) { + private void deleteSubscription(String topic, Function.ConsumerSpec consumerSpec, + String subscriptionName, String msg) { try { Actions.newBuilder() .addAction( @@ -479,8 +510,9 @@ private String getDownloadPackagePath(FunctionMetaData functionMetaData, int ins File.separatorChar); } - private File getBuiltinArchive(FunctionDetails.Builder functionDetails) throws IOException, ClassNotFoundException { - if (functionDetails.hasSource()) { + private File getBuiltinArchive(FunctionDetails.ComponentType componentType, FunctionDetails.Builder functionDetails) + throws IOException, ClassNotFoundException { + if (componentType == FunctionDetails.ComponentType.SOURCE && functionDetails.hasSource()) { SourceSpec sourceSpec = functionDetails.getSource(); if (!StringUtils.isEmpty(sourceSpec.getBuiltin())) { Connector connector = connectorsManager.getConnector(sourceSpec.getBuiltin()); @@ -495,7 +527,7 @@ private File getBuiltinArchive(FunctionDetails.Builder functionDetails) throws I } } - if (functionDetails.hasSink()) { + if (componentType == FunctionDetails.ComponentType.SINK && functionDetails.hasSink()) { SinkSpec sinkSpec = functionDetails.getSink(); if (!StringUtils.isEmpty(sinkSpec.getBuiltin())) { Connector connector = connectorsManager.getConnector(sinkSpec.getBuiltin()); @@ -511,7 +543,8 @@ private File getBuiltinArchive(FunctionDetails.Builder functionDetails) throws I } } - if (!StringUtils.isEmpty(functionDetails.getBuiltin())) { + if (componentType == FunctionDetails.ComponentType.FUNCTION + && !StringUtils.isEmpty(functionDetails.getBuiltin())) { return functionsManager.getFunctionArchive(functionDetails.getBuiltin()).toFile(); } @@ -550,21 +583,21 @@ private void fillSinkTypeClass(FunctionDetails.Builder functionDetails, } } - private static String getDownloadFileName(FunctionDetails FunctionDetails, + private static String getDownloadFileName(FunctionDetails functionDetails, Function.PackageLocationMetaData packageLocation) { if (!org.apache.commons.lang.StringUtils.isEmpty(packageLocation.getOriginalFileName())) { return packageLocation.getOriginalFileName(); } - String[] hierarchy = FunctionDetails.getClassName().split("\\."); + String[] hierarchy = functionDetails.getClassName().split("\\."); String fileName; if (hierarchy.length <= 0) { - fileName = FunctionDetails.getClassName(); + fileName = functionDetails.getClassName(); } else if (hierarchy.length == 1) { fileName = hierarchy[0]; } else { fileName = hierarchy[hierarchy.length - 2]; } - switch (FunctionDetails.getRuntime()) { + switch (functionDetails.getRuntime()) { case JAVA: return fileName + ".jar"; case PYTHON: @@ -572,7 +605,7 @@ private static String getDownloadFileName(FunctionDetails FunctionDetails, case GO: return fileName + ".go"; default: - throw new RuntimeException("Unknown runtime " + FunctionDetails.getRuntime()); + throw new RuntimeException("Unknown runtime " + functionDetails.getRuntime()); } } @@ -590,13 +623,15 @@ private void setupBatchSource(Function.FunctionDetails functionDetails) { Actions.newBuilder() .addAction( Actions.Action.builder() - .actionName(String.format("Creating intermediate topic %s with subscription %s for Batch Source %s", + .actionName(String.format("Creating intermediate topic" + + "%s with subscription %s for Batch Source %s", intermediateTopicName, intermediateTopicSubscription, fqfn)) .numRetries(10) .sleepBetweenInvocationsMs(1000) .supplier(() -> { try { - pulsarAdmin.topics().createSubscription(intermediateTopicName, intermediateTopicSubscription, MessageId.latest); + pulsarAdmin.topics().createSubscription(intermediateTopicName, + intermediateTopicSubscription, MessageId.latest); return Actions.ActionResult.builder() .success(true) .build(); @@ -624,7 +659,8 @@ private void setupBatchSource(Function.FunctionDetails functionDetails) { private void cleanupBatchSource(Function.FunctionDetails functionDetails) { if (isBatchSource(functionDetails)) { // clean up intermediate topic - String intermediateTopicName = SourceConfigUtils.computeBatchSourceIntermediateTopicName(functionDetails.getTenant(), + String intermediateTopicName = SourceConfigUtils + .computeBatchSourceIntermediateTopicName(functionDetails.getTenant(), functionDetails.getNamespace(), functionDetails.getName()).toString(); String intermediateTopicSubscription = SourceConfigUtils.computeBatchSourceInstanceSubscriptionName( functionDetails.getTenant(), functionDetails.getNamespace(), functionDetails.getName()); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java index 9d1d1233b3024..5c90b5e845753 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java @@ -210,7 +210,8 @@ public FunctionRuntimeManager(WorkerConfig workerConfig, PulsarWorkerService wor functionAuthProvider, runtimeCustomizer); this.functionActioner = new FunctionActioner(this.workerConfig, runtimeFactory, - dlogNamespace, connectorsManager, functionsManager, workerService.getBrokerAdmin()); + dlogNamespace, connectorsManager, functionsManager, workerService.getBrokerAdmin(), + workerService.getPackageUrlValidator()); this.membershipManager = membershipManager; this.functionMetaDataManager = functionMetaDataManager; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java new file mode 100644 index 0000000000000..43c90c7b2b201 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.functions.worker; + +import java.net.URI; +import java.nio.file.FileSystems; +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.pulsar.functions.proto.Function; + +/** + * Validates package URLs for functions and connectors. + * Validates that the package URL is either a file in the connectors or functions directory + * when referencing connector or function files is enabled, or matches one of the additional url patterns. + */ +public class PackageUrlValidator { + private final Path connectionsDirectory; + private final Path functionsDirectory; + private final List additionalConnectionsPatterns; + private final List additionalFunctionsPatterns; + + public PackageUrlValidator(WorkerConfig workerConfig) { + this.connectionsDirectory = resolveDirectory(workerConfig.getEnableReferencingConnectorDirectoryFiles(), + workerConfig.getConnectorsDirectory()); + this.functionsDirectory = resolveDirectory(workerConfig.getEnableReferencingFunctionsDirectoryFiles(), + workerConfig.getFunctionsDirectory()); + this.additionalConnectionsPatterns = + compilePatterns(workerConfig.getAdditionalEnabledConnectorUrlPatterns()); + this.additionalFunctionsPatterns = + compilePatterns(workerConfig.getAdditionalEnabledFunctionsUrlPatterns()); + } + + private static Path resolveDirectory(Boolean enabled, String directory) { + return enabled != null && enabled + ? FileSystems.getDefault().getPath(directory).normalize().toAbsolutePath() : null; + } + + private static List compilePatterns(List additionalPatterns) { + return additionalPatterns != null ? additionalPatterns.stream().map(Pattern::compile).collect( + Collectors.toList()) : Collections.emptyList(); + } + + boolean isValidFunctionsPackageUrl(URI functionPkgUrl) { + return doesMatch(functionPkgUrl, functionsDirectory, additionalFunctionsPatterns); + } + + boolean isValidConnectionsPackageUrl(URI functionPkgUrl) { + return doesMatch(functionPkgUrl, connectionsDirectory, additionalConnectionsPatterns); + } + + private boolean doesMatch(URI functionPkgUrl, Path directory, List patterns) { + if (directory != null && "file".equals(functionPkgUrl.getScheme())) { + Path filePath = FileSystems.getDefault().getPath(functionPkgUrl.getPath()).normalize().toAbsolutePath(); + if (filePath.startsWith(directory)) { + return true; + } + } + String functionPkgUrlString = functionPkgUrl.normalize().toString(); + for (Pattern pattern : patterns) { + if (pattern.matcher(functionPkgUrlString).matches()) { + return true; + } + } + return false; + } + + public boolean isValidPackageUrl(Function.FunctionDetails.ComponentType componentType, String functionPkgUrl) { + URI uri = URI.create(functionPkgUrl); + if (componentType == null) { + // if component type is not specified, we need to check both functions and connections + return isValidFunctionsPackageUrl(uri) || isValidConnectionsPackageUrl(uri); + } + switch (componentType) { + case FUNCTION: + return isValidFunctionsPackageUrl(uri); + case SINK: + case SOURCE: + return isValidConnectionsPackageUrl(uri); + default: + throw new IllegalArgumentException("Unknown component type: " + componentType); + } + } +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 9a1f258de720d..d6bffc84c0fef 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -119,7 +119,8 @@ public interface PulsarClientCreator { private Sinks sinks; private Sources sources; private Workers workers; - + @Getter + private PackageUrlValidator packageUrlValidator; private final PulsarClientCreator clientCreator; public PulsarWorkerService() { @@ -201,6 +202,7 @@ public void init(WorkerConfig workerConfig, this.sinks = new SinksImpl(() -> PulsarWorkerService.this); this.sources = new SourcesImpl(() -> PulsarWorkerService.this); this.workers = new WorkerImpl(() -> PulsarWorkerService.this); + this.packageUrlValidator = new PackageUrlValidator(workerConfig); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index fa2d55aef4352..66484d606d2d0 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -1301,11 +1301,17 @@ private StreamingOutput getStreamingOutput(String pkgPath) { private StreamingOutput getStreamingOutput(String pkgPath, FunctionDetails.ComponentType componentType) { return output -> { if (pkgPath.startsWith(Utils.HTTP)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, pkgPath)) { + throw new IllegalArgumentException("Invalid package url: " + pkgPath); + } URL url = URI.create(pkgPath).toURL(); try (InputStream inputStream = url.openStream()) { IOUtils.copy(inputStream, output); } } else if (pkgPath.startsWith(Utils.FILE)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, pkgPath)) { + throw new IllegalArgumentException("Invalid package url: " + pkgPath); + } URI url = URI.create(pkgPath); File file = new File(url.getPath()); Files.copy(file.toPath(), output); @@ -1715,12 +1721,17 @@ static File downloadPackageFile(PulsarWorkerService worker, String packageName) return file; } - protected File getPackageFile(String functionPkgUrl, String existingPackagePath, InputStream uploadedInputStream) + protected File getPackageFile(FunctionDetails.ComponentType componentType, String functionPkgUrl, + String existingPackagePath, InputStream uploadedInputStream) throws IOException, PulsarAdminException { File componentPackageFile = null; if (isNotBlank(functionPkgUrl)) { - componentPackageFile = getPackageFile(functionPkgUrl); + componentPackageFile = getPackageFile(componentType, functionPkgUrl); } else if (existingPackagePath.startsWith(Utils.FILE) || existingPackagePath.startsWith(Utils.HTTP)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, functionPkgUrl)) { + throw new IllegalArgumentException("Function Package url is not valid." + + "supported url (http/https/file)"); + } try { componentPackageFile = FunctionCommon.extractFileFromPkgURL(existingPackagePath); } catch (Exception e) { @@ -1729,7 +1740,7 @@ protected File getPackageFile(String functionPkgUrl, String existingPackagePath, ComponentTypeUtils.toString(componentType), functionPkgUrl)); } } else if (Utils.hasPackageTypePrefix(existingPackagePath)) { - componentPackageFile = getPackageFile(existingPackagePath); + componentPackageFile = getPackageFile(componentType, existingPackagePath); } else if (uploadedInputStream != null) { componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); } else if (!existingPackagePath.startsWith(Utils.BUILTIN)) { @@ -1741,15 +1752,12 @@ protected File getPackageFile(String functionPkgUrl, String existingPackagePath, return componentPackageFile; } - protected File downloadPackageFile(String packageName) throws IOException, PulsarAdminException { - return downloadPackageFile(worker(), packageName); - } - - protected File getPackageFile(String functionPkgUrl) throws IOException, PulsarAdminException { + protected File getPackageFile(FunctionDetails.ComponentType componentType, String functionPkgUrl) + throws IOException, PulsarAdminException { if (Utils.hasPackageTypePrefix(functionPkgUrl)) { - return downloadPackageFile(functionPkgUrl); + return downloadPackageFile(worker(), functionPkgUrl); } else { - if (!Utils.isFunctionPackageUrlSupported(functionPkgUrl)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, functionPkgUrl)) { throw new IllegalArgumentException("Function Package url is not valid." + "supported url (http/https/file)"); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index c94964897dc9a..68d4cc7cb0020 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -53,7 +53,6 @@ import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.ComponentTypeUtils; -import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; @@ -139,29 +138,13 @@ public void registerFunction(final String tenant, } Function.FunctionDetails functionDetails; - boolean isPkgUrlProvided = isNotBlank(functionPkgUrl); File componentPackageFile = null; try { // validate parameters try { - if (isPkgUrlProvided) { - if (Utils.hasPackageTypePrefix(functionPkgUrl)) { - componentPackageFile = downloadPackageFile(functionPkgUrl); - } else { - if (!Utils.isFunctionPackageUrlSupported(functionPkgUrl)) { - throw new IllegalArgumentException("Function Package url is not valid." - + "supported url (http/https/file)"); - } - try { - - componentPackageFile = FunctionCommon.extractFileFromPkgURL(functionPkgUrl); - } catch (Exception e) { - throw new IllegalArgumentException(String.format("Encountered error \"%s\" " - + "when getting %s package from %s", e.getMessage(), - ComponentTypeUtils.toString(componentType), functionPkgUrl), e); - } - } + if (isNotBlank(functionPkgUrl)) { + componentPackageFile = getPackageFile(componentType, functionPkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, functionConfig, componentPackageFile); } else { @@ -320,54 +303,18 @@ public void updateFunction(final String tenant, // validate parameters try { - if (isNotBlank(functionPkgUrl)) { - if (Utils.hasPackageTypePrefix(functionPkgUrl)) { - componentPackageFile = downloadPackageFile(functionPkgUrl); - } else { - try { - componentPackageFile = FunctionCommon.extractFileFromPkgURL(functionPkgUrl); - } catch (Exception e) { - throw new IllegalArgumentException(String.format("Encountered error \"%s\" " - + "when getting %s package from %s", e.getMessage(), - ComponentTypeUtils.toString(componentType), functionPkgUrl)); - } - } - functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, - mergedConfig, componentPackageFile); - - } else if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.FILE) - || existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.HTTP)) { - try { - componentPackageFile = FunctionCommon.extractFileFromPkgURL( - existingComponent.getPackageLocation().getPackagePath()); - } catch (Exception e) { - throw new IllegalArgumentException(String.format("Encountered error \"%s\" " - + "when getting %s package from %s", e.getMessage(), - ComponentTypeUtils.toString(componentType), functionPkgUrl)); - } - functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, - mergedConfig, componentPackageFile); - } else if (uploadedInputStream != null) { - - componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); - functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, - mergedConfig, componentPackageFile); - - } else if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.BUILTIN)) { - functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, - mergedConfig, componentPackageFile); - if (!isFunctionCodeBuiltin(functionDetails) - && (componentPackageFile == null || fileDetail == null)) { - throw new IllegalArgumentException(ComponentTypeUtils.toString(componentType) - + " Package is not provided"); - } - } else { - componentPackageFile = FunctionCommon.createPkgTempFile(); - componentPackageFile.deleteOnExit(); - WorkerUtils.downloadFromBookkeeper(worker().getDlogNamespace(), - componentPackageFile, existingComponent.getPackageLocation().getPackagePath()); - functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, - mergedConfig, componentPackageFile); + componentPackageFile = getPackageFile( + componentType, + functionPkgUrl, + existingComponent.getPackageLocation().getPackagePath(), + uploadedInputStream); + functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, + mergedConfig, componentPackageFile); + if (existingComponent.getPackageLocation().getPackagePath().startsWith(Utils.BUILTIN) + && !isFunctionCodeBuiltin(functionDetails) + && (componentPackageFile == null || fileDetail == null)) { + throw new IllegalArgumentException(ComponentTypeUtils.toString(componentType) + + " Package is not provided"); } } catch (Exception e) { log.error("Invalid update {} request @ /{}/{}/{}", ComponentTypeUtils.toString(componentType), @@ -772,7 +719,7 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (archive.startsWith(org.apache.pulsar.common.functions.Utils.BUILTIN)) { archive = archive.replaceFirst("^builtin://", ""); - FunctionsManager functionsManager = this.worker().getFunctionsManager(); + FunctionsManager functionsManager = worker().getFunctionsManager(); FunctionArchive function = functionsManager.getFunction(archive); // check if builtin function exists diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java index 8cd020b6b333b..69998ced75f4a 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java @@ -143,7 +143,7 @@ public void registerSink(final String tenant, // validate parameters try { if (isNotBlank(sinkPkgUrl)) { - componentPackageFile = getPackageFile(sinkPkgUrl); + componentPackageFile = getPackageFile(componentType, sinkPkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, sinkConfig, componentPackageFile); } else { @@ -305,6 +305,7 @@ public void updateSink(final String tenant, // validate parameters try { componentPackageFile = getPackageFile( + componentType, sinkPkgUrl, existingComponent.getPackageLocation().getPackagePath(), uploadedInputStream); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java index 59bc5888a3403..4acd673b58467 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java @@ -143,7 +143,7 @@ public void registerSource(final String tenant, // validate parameters try { if (isPkgUrlProvided) { - componentPackageFile = getPackageFile(sourcePkgUrl); + componentPackageFile = getPackageFile(componentType, sourcePkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, sourceConfig, componentPackageFile); } else { @@ -304,6 +304,7 @@ public void updateSource(final String tenant, // validate parameters try { componentPackageFile = getPackageFile( + componentType, sourcePkgUrl, existingComponent.getPackageLocation().getPackagePath(), uploadedInputStream); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java index 7502e247d9908..fa79d06bbd4a7 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.functions.worker; +import static org.apache.pulsar.common.functions.Utils.FILE; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; @@ -26,7 +27,12 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; - +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.AssertJUnit.fail; +import java.util.Arrays; +import java.util.Map; +import java.util.Optional; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.pulsar.client.admin.Packages; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -37,18 +43,10 @@ import org.apache.pulsar.functions.runtime.Runtime; import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.runtime.RuntimeSpawner; -import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig; import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; +import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactoryConfig; import org.testng.annotations.Test; -import java.util.Map; -import java.util.Optional; - -import static org.apache.pulsar.common.functions.Utils.FILE; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNull; -import static org.testng.AssertJUnit.fail; - /** * Unit test of {@link FunctionActioner}. */ @@ -67,7 +65,8 @@ public void testStartFunctionWithDLNamespace() throws Exception { workerConfig.setFunctionRuntimeFactoryClassName(ThreadRuntimeFactory.class.getName()); workerConfig.setFunctionRuntimeFactoryConfigs( ObjectMapperFactory.getThreadLocal().convertValue( - new ThreadRuntimeFactoryConfig().setThreadGroupName("test"), Map.class)); workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); + new ThreadRuntimeFactoryConfig().setThreadGroupName("test"), Map.class)); + workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); @@ -79,8 +78,8 @@ public void testStartFunctionWithDLNamespace() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class)); - Runtime runtime = mock(Runtime.class); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class), + mock(PackageUrlValidator.class)); Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder() .setFunctionDetails(Function.FunctionDetails.newBuilder().setTenant("test-tenant") .setNamespace("test-namespace").setName("func-1")) @@ -112,6 +111,8 @@ public void testStartFunctionWithPkgUrl() throws Exception { workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(Arrays.asList("file:///user/.*", "http://invalid/.*")); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(Arrays.asList("file:///user/.*", "http://invalid/.*")); String downloadDir = this.getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); workerConfig.setDownloadDirectory(downloadDir); @@ -125,44 +126,44 @@ public void testStartFunctionWithPkgUrl() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class)); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class), + new PackageUrlValidator(workerConfig)); // (1) test with file url. functionActioner should be able to consider file-url and it should be able to call // RuntimeSpawner - String pkgPathLocation = FILE + ":/user/my-file.jar"; - Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder() - .setFunctionDetails(Function.FunctionDetails.newBuilder().setTenant("test-tenant") - .setNamespace("test-namespace").setName("func-1")) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath(pkgPathLocation).build()) - .build(); - Function.Instance instance = Function.Instance.newBuilder().setFunctionMetaData(function1).setInstanceId(0) - .build(); - FunctionRuntimeInfo functionRuntimeInfo = mock(FunctionRuntimeInfo.class); - doReturn(instance).when(functionRuntimeInfo).getFunctionInstance(); - - actioner.startFunction(functionRuntimeInfo); + String pkgPathLocation = FILE + ":///user/my-file.jar"; + startFunction(actioner, pkgPathLocation); verify(runtime, times(1)).start(); // (2) test with http-url, downloading file from http should fail with UnknownHostException due to invalid url - pkgPathLocation = "http://invalid/my-file.jar"; - function1 = Function.FunctionMetaData.newBuilder() - .setFunctionDetails(Function.FunctionDetails.newBuilder().setTenant("test-tenant") - .setNamespace("test-namespace").setName("func-1")) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath(pkgPathLocation).build()) - .build(); - instance = Function.Instance.newBuilder().setFunctionMetaData(function1).setInstanceId(0).build(); - functionRuntimeInfo = mock(FunctionRuntimeInfo.class); - doReturn(instance).when(functionRuntimeInfo).getFunctionInstance(); - doThrow(new IllegalStateException("StartupException")).when(functionRuntimeInfo).setStartupException(any()); + String invalidPkgPathLocation = "http://invalid/my-file.jar"; try { - actioner.startFunction(functionRuntimeInfo); + startFunction(actioner, invalidPkgPathLocation); fail(); } catch (IllegalStateException ex) { assertEquals(ex.getMessage(), "StartupException"); } } + private void startFunction(FunctionActioner actioner, String pkgPathLocation) { + PackageLocationMetaData packageLocation = PackageLocationMetaData.newBuilder() + .setPackagePath(pkgPathLocation) + .build(); + Function.FunctionMetaData function = Function.FunctionMetaData.newBuilder() + .setFunctionDetails(Function.FunctionDetails.newBuilder().setTenant("test-tenant") + .setNamespace("test-namespace").setName("func-1")) + .setPackageLocation(packageLocation) + .build(); + Function.Instance instance = Function.Instance.newBuilder().setFunctionMetaData(function).setInstanceId(0) + .build(); + FunctionRuntimeInfo functionRuntimeInfo = mock(FunctionRuntimeInfo.class); + doReturn(instance).when(functionRuntimeInfo).getFunctionInstance(); + doThrow(new IllegalStateException("StartupException")).when(functionRuntimeInfo).setStartupException(any()); + + actioner.startFunction(functionRuntimeInfo); + } + @Test public void testFunctionAuthDisabled() throws Exception { WorkerConfig workerConfig = new WorkerConfig(); @@ -187,7 +188,8 @@ public void testFunctionAuthDisabled() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class)); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class), + mock(PackageUrlValidator.class)); String pkgPathLocation = "http://invalid/my-file.jar"; @@ -250,22 +252,13 @@ public void testStartFunctionWithPackageUrl() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), pulsarAdmin); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), pulsarAdmin, + mock(PackageUrlValidator.class)); // (1) test with file url. functionActioner should be able to consider file-url and it should be able to call // RuntimeSpawner String pkgPathLocation = "function://public/default/test-function@latest"; - Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder() - .setFunctionDetails(Function.FunctionDetails.newBuilder().setTenant("test-tenant") - .setNamespace("test-namespace").setName("func-1")) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath(pkgPathLocation).build()) - .build(); - Function.Instance instance = Function.Instance.newBuilder().setFunctionMetaData(function1).setInstanceId(0) - .build(); - FunctionRuntimeInfo functionRuntimeInfo = mock(FunctionRuntimeInfo.class); - doReturn(instance).when(functionRuntimeInfo).getFunctionInstance(); - - actioner.startFunction(functionRuntimeInfo); + startFunction(actioner, pkgPathLocation); verify(runtime, times(1)).start(); } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java index 0ab0b2728b829..1722546ea486b 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java @@ -708,7 +708,7 @@ public void testExternallyManagedRuntimeUpdate() throws Exception { FunctionActioner functionActioner = spy(new FunctionActioner( workerConfig, - kubernetesRuntimeFactory, null, null, null, null)); + kubernetesRuntimeFactory, null, null, null, null, workerService.getPackageUrlValidator())); try (final MockedStatic runtimeFactoryMockedStatic = Mockito.mockStatic(RuntimeFactory.class);) { runtimeFactoryMockedStatic.when(() -> RuntimeFactory.getFuntionRuntimeFactory(anyString())) diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java index d6aa13dc2bfb9..827eb2491b4ab 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java @@ -59,7 +59,6 @@ import org.testng.annotations.Test; public abstract class AbstractFunctionApiResourceTest extends AbstractFunctionsResourceTest { - @Test public void testListFunctionsSuccess() { mockInstanceUtils(); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java index 9da30128f1c22..893a7e1e06ec0 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java @@ -28,10 +28,12 @@ import java.io.IOException; import java.io.InputStream; import java.io.UncheckedIOException; +import java.lang.reflect.Method; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.nio.file.StandardCopyOption; +import java.util.Arrays; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -63,6 +65,7 @@ import org.apache.pulsar.functions.worker.FunctionRuntimeManager; import org.apache.pulsar.functions.worker.FunctionsManager; import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PackageUrlValidator; import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; @@ -141,7 +144,7 @@ public static File getPulsarApiExamplesNar() { } @BeforeMethod - public final void setup() throws Exception { + public final void setup(Method method) throws Exception { this.mockedManager = mock(FunctionMetaDataManager.class); this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); this.mockedRuntimeFactory = mock(RuntimeFactory.class); @@ -181,21 +184,32 @@ public final void setup() throws Exception { }).when(mockedPackages).download(any(), any()); // worker config + List urlPatterns = + Arrays.asList("http://localhost.*", "file:.*", "https://repo1.maven.org/maven2/org/apache/pulsar/.*"); WorkerConfig workerConfig = new WorkerConfig() .setWorkerId("test") .setWorkerPort(8080) .setFunctionMetadataTopicName("pulsar/functions") .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); + .setPulsarServiceUrl("pulsar://localhost:6650/") + .setAdditionalEnabledFunctionsUrlPatterns(urlPatterns) + .setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + customizeWorkerConfig(workerConfig, method); tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); + PackageUrlValidator packageUrlValidator = new PackageUrlValidator(workerConfig); + when(mockedWorkerService.getPackageUrlValidator()).thenReturn(packageUrlValidator); doSetup(); } + protected void customizeWorkerConfig(WorkerConfig workerConfig, Method method) { + + } + protected File getDefaultNarFile() { return getPulsarIOTwitterNar(); } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index a9927514efd05..64c027d73ca12 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -1380,17 +1380,15 @@ public void testRegisterSinkSuccessK8sNoUpload() throws Exception { SinkConfig sinkConfig = createDefaultSinkConfig(); sinkConfig.setArchive("builtin://cassandra"); - try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { - resource.registerSink( - tenant, - namespace, - sink, - inputStream, - mockedFormData, - null, - sinkConfig, - null); - } + resource.registerSink( + tenant, + namespace, + sink, + null, + mockedFormData, + null, + sinkConfig, + null); } /* @@ -1420,21 +1418,19 @@ public void testRegisterSinkSuccessK8sWithUpload() throws Exception { SinkConfig sinkConfig = createDefaultSinkConfig(); sinkConfig.setArchive("builtin://cassandra"); - try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { - try { - resource.registerSink( - tenant, - namespace, - sink, - inputStream, - mockedFormData, - null, - sinkConfig, - null); - Assert.fail(); - } catch (RuntimeException e) { - Assert.assertEquals(e.getMessage(), injectedErrMsg); - } + try { + resource.registerSink( + tenant, + namespace, + sink, + null, + mockedFormData, + null, + sinkConfig, + null); + Assert.fail(); + } catch (RuntimeException e) { + Assert.assertEquals(e.getMessage(), injectedErrMsg); } } From cbf3e215440573e66ef21e0c470131e87791a798 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 28 Feb 2024 11:40:21 +0200 Subject: [PATCH 02/16] [fix][sec] Upgrade Jetty to 9.4.54.v20240208 to address CVE-2024-22201 (#22144) (cherry picked from commit e3a081e4c5ea380eb505751193bc71dd0ae39281) --- .../server/src/assemble/LICENSE.bin.txt | 38 +++++++++---------- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 32 ++++++++-------- 3 files changed, 36 insertions(+), 36 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c9a41c7af5f3e..e10bc19bb66f9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -429,25 +429,25 @@ The Apache Software License, Version 2.0 - org.asynchttpclient-async-http-client-2.12.1.jar - org.asynchttpclient-async-http-client-netty-utils-2.12.1.jar * Jetty - - org.eclipse.jetty-jetty-client-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-continuation-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-http-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-io-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-proxy-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-security-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-server-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-servlet-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-servlets-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-util-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-util-ajax-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-api-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-client-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-common-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-server-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-servlet-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-alpn-server-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-client-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-continuation-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-http-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-io-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-proxy-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-security-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-server-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-servlet-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-servlets-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-util-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-util-ajax-9.4.54.v20240208.jar + - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.54.v20240208.jar + - org.eclipse.jetty.websocket-websocket-api-9.4.54.v20240208.jar + - org.eclipse.jetty.websocket-websocket-client-9.4.54.v20240208.jar + - org.eclipse.jetty.websocket-websocket-common-9.4.54.v20240208.jar + - org.eclipse.jetty.websocket-websocket-server-9.4.54.v20240208.jar + - org.eclipse.jetty.websocket-websocket-servlet-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-alpn-server-9.4.54.v20240208.jar * SnakeYaml -- org.yaml-snakeyaml-2.0.jar * RocksDB - org.rocksdb-rocksdbjni-6.29.4.1.jar * Google Error Prone Annotations - com.google.errorprone-error_prone_annotations-2.5.1.jar diff --git a/pom.xml b/pom.xml index 6eb45eb86207e..1de3ba2270b5e 100644 --- a/pom.xml +++ b/pom.xml @@ -111,7 +111,7 @@ flexible messaging model and an intuitive client API. 3.2.5 5.1.0 4.1.100.Final - 9.4.53.v20231009 + 9.4.54.v20240208 2.5.2 2.34 1.10.50 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 4095ec865ea9b..f8d79f6b01379 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -274,22 +274,22 @@ The Apache Software License, Version 2.0 - joda-time-2.10.5.jar - failsafe-2.4.4.jar * Jetty - - http2-client-9.4.53.v20231009.jar - - http2-common-9.4.53.v20231009.jar - - http2-hpack-9.4.53.v20231009.jar - - http2-http-client-transport-9.4.53.v20231009.jar - - jetty-alpn-client-9.4.53.v20231009.jar - - http2-server-9.4.53.v20231009.jar - - jetty-alpn-java-client-9.4.53.v20231009.jar - - jetty-client-9.4.53.v20231009.jar - - jetty-http-9.4.53.v20231009.jar - - jetty-io-9.4.53.v20231009.jar - - jetty-jmx-9.4.53.v20231009.jar - - jetty-security-9.4.53.v20231009.jar - - jetty-server-9.4.53.v20231009.jar - - jetty-servlet-9.4.53.v20231009.jar - - jetty-util-9.4.53.v20231009.jar - - jetty-util-ajax-9.4.53.v20231009.jar + - http2-client-9.4.54.v20240208.jar + - http2-common-9.4.54.v20240208.jar + - http2-hpack-9.4.54.v20240208.jar + - http2-http-client-transport-9.4.54.v20240208.jar + - jetty-alpn-client-9.4.54.v20240208.jar + - http2-server-9.4.54.v20240208.jar + - jetty-alpn-java-client-9.4.54.v20240208.jar + - jetty-client-9.4.54.v20240208.jar + - jetty-http-9.4.54.v20240208.jar + - jetty-io-9.4.54.v20240208.jar + - jetty-jmx-9.4.54.v20240208.jar + - jetty-security-9.4.54.v20240208.jar + - jetty-server-9.4.54.v20240208.jar + - jetty-servlet-9.4.54.v20240208.jar + - jetty-util-9.4.54.v20240208.jar + - jetty-util-ajax-9.4.54.v20240208.jar * Apache BVal - bval-jsr-2.0.0.jar * Bytecode From 9c29b76ff2be865429ad44df8683aec80deacfba Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 5 Mar 2024 15:57:07 +0800 Subject: [PATCH 03/16] Release 2.10.6 --- bouncy-castle/bc/pom.xml | 2 +- bouncy-castle/bcfips-include-test/pom.xml | 2 +- bouncy-castle/bcfips/pom.xml | 2 +- bouncy-castle/pom.xml | 2 +- buildtools/pom.xml | 2 +- distribution/io/pom.xml | 2 +- distribution/offloaders/pom.xml | 2 +- distribution/pom.xml | 2 +- distribution/server/pom.xml | 2 +- docker/pom.xml | 2 +- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- kafka-connect-avro-converter-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- pom.xml | 2 +- pulsar-broker-auth-athenz/pom.xml | 2 +- pulsar-broker-auth-sasl/pom.xml | 2 +- pulsar-broker-common/pom.xml | 2 +- pulsar-broker-shaded/pom.xml | 2 +- pulsar-broker/pom.xml | 2 +- pulsar-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml | 2 +- pulsar-client-admin-api/pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 2 +- pulsar-client-admin/pom.xml | 2 +- pulsar-client-all/pom.xml | 2 +- pulsar-client-api/pom.xml | 2 +- pulsar-client-auth-athenz/pom.xml | 2 +- pulsar-client-auth-sasl/pom.xml | 2 +- pulsar-client-messagecrypto-bc/pom.xml | 2 +- pulsar-client-shaded/pom.xml | 2 +- pulsar-client-tools-test/pom.xml | 2 +- pulsar-client-tools/pom.xml | 2 +- pulsar-client/pom.xml | 2 +- pulsar-common/pom.xml | 2 +- pulsar-config-validation/pom.xml | 2 +- pulsar-functions/api-java/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/java-examples/pom.xml | 2 +- pulsar-functions/localrun-shaded/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/pom.xml | 2 +- pulsar-functions/proto/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-functions/runtime/pom.xml | 2 +- pulsar-functions/secrets/pom.xml | 2 +- pulsar-functions/utils/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- pulsar-io/aerospike/pom.xml | 2 +- pulsar-io/aws/pom.xml | 2 +- pulsar-io/batch-data-generator/pom.xml | 2 +- pulsar-io/batch-discovery-triggerers/pom.xml | 2 +- pulsar-io/canal/pom.xml | 2 +- pulsar-io/cassandra/pom.xml | 2 +- pulsar-io/common/pom.xml | 2 +- pulsar-io/core/pom.xml | 2 +- pulsar-io/data-generator/pom.xml | 2 +- pulsar-io/debezium/core/pom.xml | 2 +- pulsar-io/debezium/mongodb/pom.xml | 2 +- pulsar-io/debezium/mssql/pom.xml | 2 +- pulsar-io/debezium/mysql/pom.xml | 2 +- pulsar-io/debezium/oracle/pom.xml | 2 +- pulsar-io/debezium/pom.xml | 2 +- pulsar-io/debezium/postgres/pom.xml | 2 +- pulsar-io/docs/pom.xml | 2 +- pulsar-io/dynamodb/pom.xml | 2 +- pulsar-io/elastic-search/pom.xml | 2 +- pulsar-io/file/pom.xml | 2 +- pulsar-io/flume/pom.xml | 2 +- pulsar-io/hbase/pom.xml | 2 +- pulsar-io/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/influxdb/pom.xml | 2 +- pulsar-io/jdbc/clickhouse/pom.xml | 2 +- pulsar-io/jdbc/core/pom.xml | 2 +- pulsar-io/jdbc/mariadb/pom.xml | 2 +- pulsar-io/jdbc/pom.xml | 2 +- pulsar-io/jdbc/postgres/pom.xml | 2 +- pulsar-io/jdbc/sqlite/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor-nar/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor/pom.xml | 2 +- pulsar-io/kafka/pom.xml | 2 +- pulsar-io/kinesis/pom.xml | 2 +- pulsar-io/mongo/pom.xml | 2 +- pulsar-io/netty/pom.xml | 2 +- pulsar-io/nsq/pom.xml | 2 +- pulsar-io/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 2 +- pulsar-io/redis/pom.xml | 2 +- pulsar-io/solr/pom.xml | 2 +- pulsar-io/twitter/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-package-management/bookkeeper-storage/pom.xml | 2 +- pulsar-package-management/core/pom.xml | 2 +- pulsar-package-management/filesystem-storage/pom.xml | 2 +- pulsar-package-management/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- pulsar-sql/java-version-trim-agent/pom.xml | 2 +- pulsar-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/pom.xml | 2 +- pulsar-testclient/pom.xml | 2 +- pulsar-transaction/common/pom.xml | 2 +- pulsar-transaction/coordinator/pom.xml | 2 +- pulsar-transaction/pom.xml | 2 +- pulsar-websocket/pom.xml | 2 +- structured-event-log/pom.xml | 2 +- testmocks/pom.xml | 2 +- tests/bc_2_0_0/pom.xml | 2 +- tests/bc_2_0_1/pom.xml | 2 +- tests/bc_2_6_0/pom.xml | 2 +- tests/docker-images/java-test-functions/pom.xml | 2 +- tests/docker-images/java-test-image/pom.xml | 2 +- tests/docker-images/latest-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/pom.xml | 2 +- tests/pulsar-client-admin-shade-test/pom.xml | 2 +- tests/pulsar-client-all-shade-test/pom.xml | 2 +- tests/pulsar-client-shade-test/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 2 +- tiered-storage/jcloud/pom.xml | 2 +- tiered-storage/pom.xml | 2 +- 126 files changed, 126 insertions(+), 126 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index c9b159abf62c1..1b5c28a6a7fd0 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index c1892ebaf9886..578e7b56c9a81 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar bouncy-castle-parent - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index e5874a8038d0f..85f34a4556909 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 7f2b96e6a9a1f..da23d3775e7e3 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 35a1db6bc60c3..37130271def02 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,7 +31,7 @@ org.apache.pulsar buildtools - 2.10.6-SNAPSHOT + 2.10.6 jar Pulsar Build Tools diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index ad0af3763c167..95cbeb9b510aa 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 504bf5fc3a4d0..4c539ee71161f 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 8c8e683097c60..69521c814e77b 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index c3254faf281f7..dca89684cf025 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/docker/pom.xml b/docker/pom.xml index 1df9ebfdd2cc6..0a5f85f6c5f8e 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index da806408d2ef5..318815f2eac14 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index c98f610b198b6..24259a73aeb06 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 7fd4e19361074..0160ae0ce66e5 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/kafka-connect-avro-converter-shaded/pom.xml b/kafka-connect-avro-converter-shaded/pom.xml index ede99b19e0621..f38bc7d0c4187 100644 --- a/kafka-connect-avro-converter-shaded/pom.xml +++ b/kafka-connect-avro-converter-shaded/pom.xml @@ -26,7 +26,7 @@ pulsar org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 8739e94bd51d2..d3411be3ad975 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pom.xml b/pom.xml index 1de3ba2270b5e..1a12ff8109d56 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 Pulsar Pulsar is a distributed pub-sub messaging platform with a very diff --git a/pulsar-broker-auth-athenz/pom.xml b/pulsar-broker-auth-athenz/pom.xml index 1b94ad5d255da..2b3540f7ee581 100644 --- a/pulsar-broker-auth-athenz/pom.xml +++ b/pulsar-broker-auth-athenz/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-broker-auth-athenz diff --git a/pulsar-broker-auth-sasl/pom.xml b/pulsar-broker-auth-sasl/pom.xml index 062830aa94bb7..6cb359b07446b 100644 --- a/pulsar-broker-auth-sasl/pom.xml +++ b/pulsar-broker-auth-sasl/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-broker-auth-sasl diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index d7a2e0556c613..e1e3d0d87bc04 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-broker-common diff --git a/pulsar-broker-shaded/pom.xml b/pulsar-broker-shaded/pom.xml index ee920cc79a90c..53a7db3cf6057 100644 --- a/pulsar-broker-shaded/pom.xml +++ b/pulsar-broker-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index b14afae47fe57..e33f5a683758b 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-1x-base/pom.xml b/pulsar-client-1x-base/pom.xml index 5c0f7c83947e8..a795409bc34a6 100644 --- a/pulsar-client-1x-base/pom.xml +++ b/pulsar-client-1x-base/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-1x-base/pulsar-client-1x/pom.xml b/pulsar-client-1x-base/pulsar-client-1x/pom.xml index 6d8b2df4d916b..0f1b7caef79ae 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-1x/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-client-1x-base - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml index 9526af73fc23b..6ab11f3a39cc4 100644 --- a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-client-1x-base - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-admin-api/pom.xml b/pulsar-client-admin-api/pom.xml index 9ff21d3fdaecb..326a6e64adab6 100644 --- a/pulsar-client-admin-api/pom.xml +++ b/pulsar-client-admin-api/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 0f73854bc7b9b..3d45d3047eeb7 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 4061ef0718d1f..51ea727388307 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 9d66319f0f6bd..0bd7e822fa668 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-api/pom.xml b/pulsar-client-api/pom.xml index 50a693334ccc1..12700ceba27cc 100644 --- a/pulsar-client-api/pom.xml +++ b/pulsar-client-api/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-auth-athenz/pom.xml b/pulsar-client-auth-athenz/pom.xml index a6defb7f34f69..523dc8c23f6cf 100644 --- a/pulsar-client-auth-athenz/pom.xml +++ b/pulsar-client-auth-athenz/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-auth-sasl/pom.xml b/pulsar-client-auth-sasl/pom.xml index 955698272031d..c78e06caece61 100644 --- a/pulsar-client-auth-sasl/pom.xml +++ b/pulsar-client-auth-sasl/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-messagecrypto-bc/pom.xml b/pulsar-client-messagecrypto-bc/pom.xml index 70f38afca1999..7f87fe122cd31 100644 --- a/pulsar-client-messagecrypto-bc/pom.xml +++ b/pulsar-client-messagecrypto-bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index cf34e6f1d5091..689733d659447 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index 85c532ac1c3bb..c723143917fee 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 2ec40e7f7b1fd..068f8c744bf54 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index b2124d9293c8d..9b19613835d5d 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 6d95bd5025867..2f3e48ce2be85 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-config-validation/pom.xml b/pulsar-config-validation/pom.xml index 080b78095bccc..c02d88d4aa1af 100644 --- a/pulsar-config-validation/pom.xml +++ b/pulsar-config-validation/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-functions/api-java/pom.xml b/pulsar-functions/api-java/pom.xml index efa510d4d105e..a9406921b1b7f 100644 --- a/pulsar-functions/api-java/pom.xml +++ b/pulsar-functions/api-java/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-api diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index ae16da8e44bfb..ee8c5374bb182 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-instance diff --git a/pulsar-functions/java-examples/pom.xml b/pulsar-functions/java-examples/pom.xml index 9eeffc0186e84..98cd1114db58f 100644 --- a/pulsar-functions/java-examples/pom.xml +++ b/pulsar-functions/java-examples/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-api-examples diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index 82a792459e259..baf2b72e00178 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index b80e369820e77..81a2550d6f196 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-functions/pom.xml b/pulsar-functions/pom.xml index 6ee467025e564..895ab9bef46d8 100644 --- a/pulsar-functions/pom.xml +++ b/pulsar-functions/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions diff --git a/pulsar-functions/proto/pom.xml b/pulsar-functions/proto/pom.xml index 4aee26b900b7c..aeeb39a5c6183 100644 --- a/pulsar-functions/proto/pom.xml +++ b/pulsar-functions/proto/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-proto diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 7544725021f0f..1045fadd29e5b 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index 051a9aacd9ed6..0154e62233eac 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-runtime diff --git a/pulsar-functions/secrets/pom.xml b/pulsar-functions/secrets/pom.xml index 801a5b0848de4..4c2c201165dce 100644 --- a/pulsar-functions/secrets/pom.xml +++ b/pulsar-functions/secrets/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-secrets diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 0c08364a68d34..2a0d035a6cf0f 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 pulsar-functions-utils diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index 1476a8de4394a..f83b6b17f2c12 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-io/aerospike/pom.xml b/pulsar-io/aerospike/pom.xml index 549067c8ffe36..771c909507b43 100644 --- a/pulsar-io/aerospike/pom.xml +++ b/pulsar-io/aerospike/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-aerospike diff --git a/pulsar-io/aws/pom.xml b/pulsar-io/aws/pom.xml index 724835d80f8ad..d2c42408cc2d2 100644 --- a/pulsar-io/aws/pom.xml +++ b/pulsar-io/aws/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-aws diff --git a/pulsar-io/batch-data-generator/pom.xml b/pulsar-io/batch-data-generator/pom.xml index e4a8f3f8c6c15..bb4fa445bbca3 100644 --- a/pulsar-io/batch-data-generator/pom.xml +++ b/pulsar-io/batch-data-generator/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-batch-data-generator diff --git a/pulsar-io/batch-discovery-triggerers/pom.xml b/pulsar-io/batch-discovery-triggerers/pom.xml index ef14e98973c8c..82672d464cd58 100644 --- a/pulsar-io/batch-discovery-triggerers/pom.xml +++ b/pulsar-io/batch-discovery-triggerers/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-batch-discovery-triggerers diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index e61e8a0567753..bb41b142cf6c6 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-io/cassandra/pom.xml b/pulsar-io/cassandra/pom.xml index 8c796b5fb9b5e..7a8d5795bf790 100644 --- a/pulsar-io/cassandra/pom.xml +++ b/pulsar-io/cassandra/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-cassandra diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index 8185bbcfca16f..3c9bb46d20231 100644 --- a/pulsar-io/common/pom.xml +++ b/pulsar-io/common/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-common diff --git a/pulsar-io/core/pom.xml b/pulsar-io/core/pom.xml index a5d13db1df6cf..d341e110927d0 100644 --- a/pulsar-io/core/pom.xml +++ b/pulsar-io/core/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-core diff --git a/pulsar-io/data-generator/pom.xml b/pulsar-io/data-generator/pom.xml index d9445778bdd07..6a95d07e1b54d 100644 --- a/pulsar-io/data-generator/pom.xml +++ b/pulsar-io/data-generator/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-data-generator diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index de45a6171edcb..46b7cdd592d80 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium-core diff --git a/pulsar-io/debezium/mongodb/pom.xml b/pulsar-io/debezium/mongodb/pom.xml index 0f9f5cc43d86a..0a01f218a41f3 100644 --- a/pulsar-io/debezium/mongodb/pom.xml +++ b/pulsar-io/debezium/mongodb/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium-mongodb diff --git a/pulsar-io/debezium/mssql/pom.xml b/pulsar-io/debezium/mssql/pom.xml index e2d9bf0125de6..c076715fbc787 100644 --- a/pulsar-io/debezium/mssql/pom.xml +++ b/pulsar-io/debezium/mssql/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium-mssql diff --git a/pulsar-io/debezium/mysql/pom.xml b/pulsar-io/debezium/mysql/pom.xml index 05a7410c0567d..03e396cfaa8f5 100644 --- a/pulsar-io/debezium/mysql/pom.xml +++ b/pulsar-io/debezium/mysql/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium-mysql diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index d1da410bde301..d7fefcd5a0d6b 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium-oracle diff --git a/pulsar-io/debezium/pom.xml b/pulsar-io/debezium/pom.xml index 2f70452afb985..aa024b86b6521 100644 --- a/pulsar-io/debezium/pom.xml +++ b/pulsar-io/debezium/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium diff --git a/pulsar-io/debezium/postgres/pom.xml b/pulsar-io/debezium/postgres/pom.xml index d4c21e5e0cf7d..1645bdf1cf135 100644 --- a/pulsar-io/debezium/postgres/pom.xml +++ b/pulsar-io/debezium/postgres/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-debezium-postgres diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index 169f3bc6e0b11..13f5a001440a5 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-docs diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index b7610d30728e8..34c7f7e2961f1 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-dynamodb diff --git a/pulsar-io/elastic-search/pom.xml b/pulsar-io/elastic-search/pom.xml index 8a630fb071bef..4b00d709f1825 100644 --- a/pulsar-io/elastic-search/pom.xml +++ b/pulsar-io/elastic-search/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-elastic-search Pulsar IO :: ElasticSearch diff --git a/pulsar-io/file/pom.xml b/pulsar-io/file/pom.xml index 5cf419d077558..1b78f1bc1988b 100644 --- a/pulsar-io/file/pom.xml +++ b/pulsar-io/file/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-file diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index d73430b5f38f7..7e62c299f01cc 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-flume diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index cefe293449f83..6e9a7dc2e1418 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-hbase Pulsar IO :: Hbase diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index 4a965f7ae6ede..29a52b17ff551 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-hdfs2 Pulsar IO :: Hdfs2 diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 3eaaeb8203a2a..68327c7166a2a 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-hdfs3 Pulsar IO :: Hdfs3 diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index f4556e4702b1c..4a9ea205f6c00 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-influxdb diff --git a/pulsar-io/jdbc/clickhouse/pom.xml b/pulsar-io/jdbc/clickhouse/pom.xml index 0d99d61555223..0745d9950a76c 100644 --- a/pulsar-io/jdbc/clickhouse/pom.xml +++ b/pulsar-io/jdbc/clickhouse/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 09ffe777288d7..c2170c174e918 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-io/jdbc/mariadb/pom.xml b/pulsar-io/jdbc/mariadb/pom.xml index 8b5a42990cf00..d909af3bf165c 100644 --- a/pulsar-io/jdbc/mariadb/pom.xml +++ b/pulsar-io/jdbc/mariadb/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-io/jdbc/pom.xml b/pulsar-io/jdbc/pom.xml index 3d76adc5ae741..85e71ddd6c683 100644 --- a/pulsar-io/jdbc/pom.xml +++ b/pulsar-io/jdbc/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-jdbc diff --git a/pulsar-io/jdbc/postgres/pom.xml b/pulsar-io/jdbc/postgres/pom.xml index 701321d70b186..f9fe74a07ebfb 100644 --- a/pulsar-io/jdbc/postgres/pom.xml +++ b/pulsar-io/jdbc/postgres/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-io/jdbc/sqlite/pom.xml b/pulsar-io/jdbc/sqlite/pom.xml index bd20a5c2fd4b9..e5fca91b6c582 100644 --- a/pulsar-io/jdbc/sqlite/pom.xml +++ b/pulsar-io/jdbc/sqlite/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 pulsar-io-jdbc-sqlite diff --git a/pulsar-io/kafka-connect-adaptor-nar/pom.xml b/pulsar-io/kafka-connect-adaptor-nar/pom.xml index ef3460645df92..76d532dcdd0fd 100644 --- a/pulsar-io/kafka-connect-adaptor-nar/pom.xml +++ b/pulsar-io/kafka-connect-adaptor-nar/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-kafka-connect-adaptor-nar diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 5a03a21d85eef..94b3ac3a72c65 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-kafka-connect-adaptor diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 4aef852521682..19c6518df68b2 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-kafka diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index 7bc58f1ae81b1..74d0e57aedaf3 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-kinesis diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index 507e56723b5ee..5024127718bb3 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-mongo diff --git a/pulsar-io/netty/pom.xml b/pulsar-io/netty/pom.xml index e4d2b3d0ede76..40912b136225b 100644 --- a/pulsar-io/netty/pom.xml +++ b/pulsar-io/netty/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-netty diff --git a/pulsar-io/nsq/pom.xml b/pulsar-io/nsq/pom.xml index 1a0ea5743bfdd..2a41f92dac4a4 100644 --- a/pulsar-io/nsq/pom.xml +++ b/pulsar-io/nsq/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-nsq diff --git a/pulsar-io/pom.xml b/pulsar-io/pom.xml index 6195debcb2e7b..f0dc1551f9a9e 100644 --- a/pulsar-io/pom.xml +++ b/pulsar-io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index cb3875b5a339a..8397fc84b361a 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-rabbitmq diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 4def845713231..64ba40fb3e685 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-redis diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index f2298d54f7efa..ee5072ce2faec 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 diff --git a/pulsar-io/twitter/pom.xml b/pulsar-io/twitter/pom.xml index b568be8e7e1ea..9167b3a28f97e 100644 --- a/pulsar-io/twitter/pom.xml +++ b/pulsar-io/twitter/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6-SNAPSHOT + 2.10.6 pulsar-io-twitter diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index ec96ca7d0e2b0..3c5e0eb40ecd1 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-package-management/bookkeeper-storage/pom.xml b/pulsar-package-management/bookkeeper-storage/pom.xml index 380a17f7e9e34..1af8f528816bc 100644 --- a/pulsar-package-management/bookkeeper-storage/pom.xml +++ b/pulsar-package-management/bookkeeper-storage/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-package-management/core/pom.xml b/pulsar-package-management/core/pom.xml index d40119d1a617a..749ec9313cb3d 100644 --- a/pulsar-package-management/core/pom.xml +++ b/pulsar-package-management/core/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-package-management/filesystem-storage/pom.xml b/pulsar-package-management/filesystem-storage/pom.xml index 365159477beec..3958ce4c313b2 100644 --- a/pulsar-package-management/filesystem-storage/pom.xml +++ b/pulsar-package-management/filesystem-storage/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-package-management/pom.xml b/pulsar-package-management/pom.xml index 9ffd136b6c6c0..5f91f6eb84ef9 100644 --- a/pulsar-package-management/pom.xml +++ b/pulsar-package-management/pom.xml @@ -25,7 +25,7 @@ pulsar org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. 4.0.0 diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 63906fb44dc27..cd408cde17020 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-proxy diff --git a/pulsar-sql/java-version-trim-agent/pom.xml b/pulsar-sql/java-version-trim-agent/pom.xml index 48c43e6dda279..cb2e6138c1ded 100644 --- a/pulsar-sql/java-version-trim-agent/pom.xml +++ b/pulsar-sql/java-version-trim-agent/pom.xml @@ -24,7 +24,7 @@ pulsar-sql org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/pulsar-sql/pom.xml b/pulsar-sql/pom.xml index f5331dae23b94..69d313469939b 100644 --- a/pulsar-sql/pom.xml +++ b/pulsar-sql/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-sql diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index 0c940e623b644..a45c4cf080682 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-sql - 2.10.6-SNAPSHOT + 2.10.6 pulsar-presto-distribution diff --git a/pulsar-sql/presto-pulsar-plugin/pom.xml b/pulsar-sql/presto-pulsar-plugin/pom.xml index 00057135aef52..6ec62caec43f8 100644 --- a/pulsar-sql/presto-pulsar-plugin/pom.xml +++ b/pulsar-sql/presto-pulsar-plugin/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-sql - 2.10.6-SNAPSHOT + 2.10.6 pulsar-presto-connector diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index 3501466a82834..bb0e06d3f2e08 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-sql - 2.10.6-SNAPSHOT + 2.10.6 pulsar-presto-connector-original diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 8b8633fb6c420..6433f9db5a176 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/pulsar-transaction/common/pom.xml b/pulsar-transaction/common/pom.xml index 308275a05b7f8..25a01608f2ce5 100644 --- a/pulsar-transaction/common/pom.xml +++ b/pulsar-transaction/common/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-transaction-parent - 2.10.6-SNAPSHOT + 2.10.6 pulsar-transaction-common diff --git a/pulsar-transaction/coordinator/pom.xml b/pulsar-transaction/coordinator/pom.xml index 79961dadbda1f..7b5d45bf1e83a 100644 --- a/pulsar-transaction/coordinator/pom.xml +++ b/pulsar-transaction/coordinator/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-transaction-parent - 2.10.6-SNAPSHOT + 2.10.6 pulsar-transaction-coordinator diff --git a/pulsar-transaction/pom.xml b/pulsar-transaction/pom.xml index 5ba989961ce80..23ed756b1ffa2 100644 --- a/pulsar-transaction/pom.xml +++ b/pulsar-transaction/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 pulsar-transaction-parent diff --git a/pulsar-websocket/pom.xml b/pulsar-websocket/pom.xml index dd7fa68812d78..12282020f1fe0 100644 --- a/pulsar-websocket/pom.xml +++ b/pulsar-websocket/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index 5d01dcca3d86e..ad12dc090defd 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/testmocks/pom.xml b/testmocks/pom.xml index 86704930e1905..a9ebf86e78027 100644 --- a/testmocks/pom.xml +++ b/testmocks/pom.xml @@ -25,7 +25,7 @@ pulsar org.apache.pulsar - 2.10.6-SNAPSHOT + 2.10.6 testmocks diff --git a/tests/bc_2_0_0/pom.xml b/tests/bc_2_0_0/pom.xml index f919d8c2e40d7..74e6b29e4b1d6 100644 --- a/tests/bc_2_0_0/pom.xml +++ b/tests/bc_2_0_0/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 bc_2_0_0 diff --git a/tests/bc_2_0_1/pom.xml b/tests/bc_2_0_1/pom.xml index f64327cc32c76..8064d106e302f 100644 --- a/tests/bc_2_0_1/pom.xml +++ b/tests/bc_2_0_1/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 bc_2_0_1 diff --git a/tests/bc_2_6_0/pom.xml b/tests/bc_2_6_0/pom.xml index c02bbe5491fcf..92680683c9e0d 100644 --- a/tests/bc_2_6_0/pom.xml +++ b/tests/bc_2_6_0/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 diff --git a/tests/docker-images/java-test-functions/pom.xml b/tests/docker-images/java-test-functions/pom.xml index 8173511dc8fc8..b385c4ab7c521 100644 --- a/tests/docker-images/java-test-functions/pom.xml +++ b/tests/docker-images/java-test-functions/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 java-test-functions diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml index 4e4e901c40689..ee5d46d41294e 100644 --- a/tests/docker-images/java-test-image/pom.xml +++ b/tests/docker-images/java-test-image/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 java-test-image diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index dbcb967073740..feab4c22c37f9 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 2.10.6-SNAPSHOT + 2.10.6 4.0.0 latest-version-image diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml index 90f71564eba32..78d96ebd53d75 100644 --- a/tests/docker-images/pom.xml +++ b/tests/docker-images/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 docker-images Apache Pulsar :: Tests :: Docker Images diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 5971840bb04c2..f4ee351072f05 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 integration diff --git a/tests/pom.xml b/tests/pom.xml index 4cdeb4753c456..aaf1eedfd1902 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 org.apache.pulsar.tests tests-parent diff --git a/tests/pulsar-client-admin-shade-test/pom.xml b/tests/pulsar-client-admin-shade-test/pom.xml index 4dace92c27303..828c0e554edd7 100644 --- a/tests/pulsar-client-admin-shade-test/pom.xml +++ b/tests/pulsar-client-admin-shade-test/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 pulsar-client-admin-shade-test diff --git a/tests/pulsar-client-all-shade-test/pom.xml b/tests/pulsar-client-all-shade-test/pom.xml index 0fab55fe122a7..9af89ea26e7d9 100644 --- a/tests/pulsar-client-all-shade-test/pom.xml +++ b/tests/pulsar-client-all-shade-test/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 pulsar-client-all-shade-test diff --git a/tests/pulsar-client-shade-test/pom.xml b/tests/pulsar-client-shade-test/pom.xml index 5922f31395524..50d7ca86e9955 100644 --- a/tests/pulsar-client-shade-test/pom.xml +++ b/tests/pulsar-client-shade-test/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6-SNAPSHOT + 2.10.6 pulsar-client-shade-test diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 664e4b289b410..73ba9a72dc01e 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar tiered-storage-parent - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 1d5b6489dc93e..e191e123b8bdf 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar tiered-storage-parent - 2.10.6-SNAPSHOT + 2.10.6 .. diff --git a/tiered-storage/pom.xml b/tiered-storage/pom.xml index a2447cf6830f7..6e4a8dc2f2b4d 100644 --- a/tiered-storage/pom.xml +++ b/tiered-storage/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6-SNAPSHOT + 2.10.6 .. From 858f04edce29569dae9d5a10f2c0b6ce2fdca79d Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sun, 23 Jul 2023 12:56:04 +0800 Subject: [PATCH 04/16] [improve][broker] Avoid print redirect exception log when get list from bundle (#20846) (cherry picked from commit 9256407cdca1ab6d9b3a59ce404dccb09953ab24) --- .../broker/admin/v2/NonPersistentTopics.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 87cc6650bb902..854f76c3abf87 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -498,19 +498,19 @@ public void getListFromBundle( } asyncResponse.resume(topicList); } catch (Exception e) { - log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), - namespaceName, bundleRange, e); + if (!isRedirectException(e)) { + log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), + namespaceName, bundleRange, e); + } asyncResponse.resume(new RestException(e)); } } }).exceptionally(ex -> { - log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), - namespaceName, bundleRange, ex); - if (ex.getCause() instanceof WebApplicationException) { - asyncResponse.resume(ex.getCause()); - } else { - asyncResponse.resume(new RestException(ex.getCause())); + if (!isRedirectException(ex)) { + log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), + namespaceName, bundleRange, ex); } + resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); } From 10cbcc01547ebac74c38150dc9e07b9e7c226104 Mon Sep 17 00:00:00 2001 From: gaozhangmin Date: Mon, 9 Jan 2023 19:39:09 +0800 Subject: [PATCH 05/16] [improve][admin]internalGetMessageById shouldn't be allowed on partitioned topic (#19013) Co-authored-by: gavingaozhangmin (cherry picked from commit b05fddb1af03456438f27217dc6979be00fac19e) --- .../admin/impl/PersistentTopicsBase.java | 107 +++++++++--------- .../broker/admin/v1/PersistentTopics.java | 20 ++-- .../broker/admin/v2/PersistentTopics.java | 20 ++-- 3 files changed, 80 insertions(+), 67 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 867f45a5e3ccc..ed3896b145612 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -2619,60 +2619,65 @@ private PositionImpl calculatePositionAckSet(boolean isExcluded, int batchSize, return seekPosition; } - protected void internalGetMessageById(AsyncResponse asyncResponse, long ledgerId, long entryId, - boolean authoritative) { - // will redirect if the topic not owned by current broker - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) - .thenCompose(__ -> { - CompletableFuture ret; - if (topicName.isGlobal()) { - ret = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret; - }) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenAccept(topic -> { - ManagedLedgerImpl ledger = - (ManagedLedgerImpl) ((PersistentTopic) topic).getManagedLedger(); - ledger.asyncReadEntry(new PositionImpl(ledgerId, entryId), - new AsyncCallbacks.ReadEntryCallback() { - @Override - public void readEntryFailed(ManagedLedgerException exception, - Object ctx) { - asyncResponse.resume(new RestException(exception)); - } + protected CompletableFuture internalGetMessageById(long ledgerId, long entryId, boolean authoritative) { + CompletableFuture future; + if (topicName.isGlobal()) { + future = validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + future = CompletableFuture.completedFuture(null); + } + return future.thenCompose(__ -> { + if (topicName.isPartitioned()) { + return CompletableFuture.completedFuture(null); + } else { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenAccept(topicMetadata -> { + if (topicMetadata.partitions > 0) { + log.warn("[{}] Not supported getMessageById operation on partitioned-topic {}", + clientAppId(), topicName); + throw new RestException(Status.METHOD_NOT_ALLOWED, + "GetMessageById is not allowed on partitioned-topic"); + } + }); - @Override - public void readEntryComplete(Entry entry, Object ctx) { - try { - asyncResponse.resume(generateResponseWithEntry(entry)); - } catch (IOException exception) { - asyncResponse.resume(new RestException(exception)); - } finally { - if (entry != null) { - entry.release(); - } - } - } + } + }) + .thenCompose(ignore -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + CompletableFuture results = new CompletableFuture<>(); + ManagedLedgerImpl ledger = + (ManagedLedgerImpl) ((PersistentTopic) topic).getManagedLedger(); + ledger.asyncReadEntry(new PositionImpl(ledgerId, entryId), + new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryFailed(ManagedLedgerException exception, + Object ctx) { + throw new RestException(exception); + } - @Override - public String toString() { - return String.format("Topic [%s] internal get message by id", - PersistentTopicsBase.this.topicName); + @Override + public void readEntryComplete(Entry entry, Object ctx) { + try { + results.complete(generateResponseWithEntry(entry)); + } catch (IOException exception) { + throw new RestException(exception); + } finally { + if (entry != null) { + entry.release(); } - }, null); - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { - log.error("[{}] Failed to get message with ledgerId {} entryId {} from {}", - clientAppId(), ledgerId, entryId, topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + } + } + + @Override + public String toString() { + return String.format("Topic [%s] internal get message by id", + PersistentTopicsBase.this.topicName); + } + }, null); + return results; + }); } protected CompletableFuture internalGetMessageIdByTimestamp(long timestamp, boolean authoritative) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index 2009de113d2a4..f246b00419d9b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -720,14 +720,18 @@ public void getMessageByID(@Suspended final AsyncResponse asyncResponse, @PathPa @PathParam("topic") @Encoded String encodedTopic, @PathParam("ledgerId") Long ledgerId, @PathParam("entryId") Long entryId, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - try { - validateTopicName(property, cluster, namespace, encodedTopic); - internalGetMessageById(asyncResponse, ledgerId, entryId, authoritative); - } catch (WebApplicationException wae) { - asyncResponse.resume(wae); - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - } + validateTopicName(property, cluster, namespace, encodedTopic); + internalGetMessageById(ledgerId, entryId, authoritative) + .thenAccept(asyncResponse::resume) + .exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isRedirectException(ex)) { + log.error("[{}] Failed to get message with ledgerId {} entryId {} from {}", + clientAppId(), ledgerId, entryId, topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index afa3f4495daa1..71c9dd6473212 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -1699,14 +1699,18 @@ public void getMessageById( @PathParam("entryId") long entryId, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - try { - validateTopicName(tenant, namespace, encodedTopic); - internalGetMessageById(asyncResponse, ledgerId, entryId, authoritative); - } catch (WebApplicationException wae) { - asyncResponse.resume(wae); - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - } + validateTopicName(tenant, namespace, encodedTopic); + internalGetMessageById(ledgerId, entryId, authoritative) + .thenAccept(asyncResponse::resume) + .exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isRedirectException(ex)) { + log.error("[{}] Failed to get message with ledgerId {} entryId {} from {}", + clientAppId(), ledgerId, entryId, topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } @GET From a080aebe015f18a1cbf94be50a57105bd096cade Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 18 Feb 2024 15:46:52 +0800 Subject: [PATCH 06/16] [improve] [broker] Do not print an Error log when responding to `HTTP-404` when calling `Admin API` and the topic does not exist. (#21995) (cherry picked from commit 48b4481969cb6028186a7a84b8be8af90770674b) --- .../pulsar/broker/admin/AdminResource.java | 11 +++ .../admin/impl/PersistentTopicsBase.java | 87 +++++++++---------- .../broker/admin/v2/NonPersistentTopics.java | 4 +- .../broker/admin/v2/PersistentTopics.java | 14 +-- .../pulsar/broker/admin/v3/Transactions.java | 16 ++++ 5 files changed, 80 insertions(+), 52 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 30526a0787d24..f64e1d94507e5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -798,6 +798,17 @@ protected static boolean isRedirectException(Throwable ex) { == Status.TEMPORARY_REDIRECT.getStatusCode(); } + protected static boolean isNotFoundException(Throwable ex) { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + return realCause instanceof WebApplicationException + && ((WebApplicationException) realCause).getResponse().getStatus() + == Status.NOT_FOUND.getStatusCode(); + } + + protected static boolean isNot307And404Exception(Throwable ex) { + return !isRedirectException(ex) && !isNotFoundException(ex); + } + protected static String getTopicNotFoundErrorMessage(String topic) { return String.format("Topic %s not found", topic); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index ed3896b145612..82711096701f2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -838,7 +838,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while unloading topic {}", clientAppId(), topicName, ex); } @@ -848,7 +848,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while unloading topic {}", clientAppId(), topicName, ex); } @@ -1058,7 +1058,7 @@ private void internalUnloadNonPartitionedTopicAsync(AsyncResponse asyncResponse, })) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1080,7 +1080,7 @@ private void internalUnloadTransactionCoordinatorAsync(AsyncResponse asyncRespon })) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to unload tc {},{}", clientAppId(), topicName.getPartitionIndex(), ex); } @@ -1206,7 +1206,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned topic metadata while get" + " subscriptions for topic {}", clientAppId(), topicName, ex); } @@ -1216,7 +1216,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace/topic ownership while get subscriptions" + " for topic {}", clientAppId(), topicName, ex); } @@ -1225,7 +1225,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1264,7 +1264,7 @@ private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncR .thenAccept(topic -> asyncResponse.resume(Lists.newArrayList(topic.getSubscriptions().keys()))) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1376,7 +1376,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while get managed info for {}", clientAppId(), topicName, ex); } @@ -1386,7 +1386,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while get managed info for {}", clientAppId(), topicName, ex); } @@ -1488,7 +1488,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1542,7 +1542,7 @@ protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1656,8 +1656,9 @@ private void internalDeleteSubscriptionForNonPartitionedTopic(AsyncResponse asyn "Subscription has active connected consumers")); } else { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, cause); + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to analyze subscription backlog {} {}", + clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); } @@ -1683,7 +1684,7 @@ private void internalUpdateSubscriptionPropertiesForNonPartitionedTopic(AsyncRes }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1850,7 +1851,7 @@ protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subNa } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1893,7 +1894,7 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1955,7 +1956,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, subName, ex); } @@ -2022,7 +2023,7 @@ protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResp ) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, ex); } @@ -2085,7 +2086,7 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription up to {} on {}", clientAppId(), expireTimeInSeconds, topicName, ex); } @@ -2290,7 +2291,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2300,7 +2301,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2431,7 +2432,7 @@ protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2512,7 +2513,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -2521,7 +2522,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -3069,7 +3070,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3079,7 +3080,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3087,7 +3088,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership to get backlog size for topic " + "{}", clientAppId(), topicName, ex); } @@ -3599,7 +3600,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3607,7 +3608,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3688,7 +3689,7 @@ protected void internalExpireMessagesByTimestamp(AsyncResponse asyncResponse, St ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, topicName, ex); } @@ -3798,7 +3799,7 @@ protected void internalExpireMessagesByPosition(AsyncResponse asyncResponse, Str }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages up to {} on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex); } @@ -3953,7 +3954,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3962,7 +3963,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership to trigger compaction on topic {}", clientAppId(), topicName, ex); } @@ -3991,7 +3992,7 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4028,7 +4029,7 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger offload for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4045,7 +4046,7 @@ protected void internalOffloadStatus(AsyncResponse asyncResponse, boolean author asyncResponse.resume(offloadProcessStatus); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to offload status on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4518,7 +4519,7 @@ protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean aut }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get last messageId {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4779,9 +4780,7 @@ protected CompletableFuture internalRemoveSubscribeRate(boolean isGlobal) protected void handleTopicPolicyException(String methodName, Throwable thr, AsyncResponse asyncResponse) { Throwable cause = thr.getCause(); - if (!(cause instanceof WebApplicationException) || !( - ((WebApplicationException) cause).getResponse().getStatus() == 307 - || ((WebApplicationException) cause).getResponse().getStatus() == 404)) { + if (isNot307And404Exception(cause)) { log.error("[{}] Failed to perform {} on topic {}", clientAppId(), methodName, topicName, cause); } @@ -4943,7 +4942,7 @@ protected void internalSetReplicatedSubscriptionStatus(AsyncResponse asyncRespon resultFuture.exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.warn("[{}] Failed to change replicated subscription status to {} - {} {}", clientAppId(), enabled, topicName, subName, ex); } @@ -4988,7 +4987,7 @@ private void internalSetReplicatedSubscriptionStatusForNonPartitionedTopic( } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to set replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } @@ -5090,7 +5089,7 @@ protected void internalGetReplicatedSubscriptionStatus(AsyncResponse asyncRespon } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 854f76c3abf87..b75ec765796e9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -498,7 +498,7 @@ public void getListFromBundle( } asyncResponse.resume(topicList); } catch (Exception e) { - if (!isRedirectException(e)) { + if (!isNot307And404Exception(e)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, e); } @@ -506,7 +506,7 @@ public void getListFromBundle( } } }).exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 71c9dd6473212..e2cfc86e8fdbd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -105,7 +105,7 @@ public void getList( internalGetListAsync(Optional.ofNullable(bundle)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get topic list {}", clientAppId(), namespaceName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -907,7 +907,7 @@ public void getProperties( internalGetPropertiesAsync(authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get topic {} properties", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1704,7 +1704,7 @@ public void getMessageById( .thenAccept(asyncResponse::resume) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get message with ledgerId {} entryId {} from {}", clientAppId(), ledgerId, entryId, topicName, ex); } @@ -1748,9 +1748,11 @@ public void getMessageIdByTimestamp( } }) .exceptionally(ex -> { - log.error("[{}] Failed to get message ID by timestamp {} from {}", - clientAppId(), timestamp, topicName, ex); - resumeAsyncResponseExceptionally(asyncResponse, ex); + if (isNot307And404Exception(ex)) { + log.error("[{}] Failed to get message ID by timestamp {} from {}", + clientAppId(), timestamp, topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + } return null; }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index 9cb825b9f8e1f..a0faeb654596a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -90,6 +90,10 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp Long.parseLong(leastSigBits)) .thenAccept(stat -> asyncResponse.resume(stat)) .exceptionally(ex -> { + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to get transaction state in transaction buffer {}", + clientAppId(), topicName, ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); @@ -124,6 +128,10 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async Long.parseLong(leastSigBits), subName) .thenAccept(stat -> asyncResponse.resume(stat)) .exceptionally(ex -> { + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to get transaction state in pending ack {}", + clientAppId(), topicName, ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); @@ -154,6 +162,10 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon internalGetTransactionBufferStats(authoritative) .thenAccept(stat -> asyncResponse.resume(stat)) .exceptionally(ex -> { + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to get transaction buffer stats in topic {}", + clientAppId(), topicName, ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); @@ -185,6 +197,10 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, internalGetPendingAckStats(authoritative, subName) .thenAccept(stats -> asyncResponse.resume(stats)) .exceptionally(ex -> { + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to get transaction pending ack stats in topic {}", + clientAppId(), topicName, ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); From cb65e17ea8a789d2b689b7b5af8500215ebfb08b Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 6 Mar 2024 22:00:45 +0800 Subject: [PATCH 07/16] [improve][broker] Consistently add fine-grain authorization to REST API (#22202) (cherry picked from commit 68c10925df43769eee7265b4af0ac8ee4913e715) --- .../broker/admin/v2/PersistentTopics.java | 187 ++++++++++------ .../broker/admin/TopicPoliciesAuthZTest.java | 206 +++++++++++++++++- 2 files changed, 331 insertions(+), 62 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index e2cfc86e8fdbd..4dd4c1310cc62 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -68,6 +68,7 @@ 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.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; @@ -347,7 +348,8 @@ public void getOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.OFFLOAD, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetOffloadPolicies(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -370,7 +372,9 @@ public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @ApiParam(value = "Offload policies for the specified topic") OffloadPoliciesImpl offloadPolicies) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperation(topicName, PolicyName.OFFLOAD, PolicyOperation.WRITE); + validateTopicPolicyOperationAsync(topicName, PolicyName.OFFLOAD, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetOffloadPolicies(offloadPolicies, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -392,7 +396,8 @@ public void removeOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.OFFLOAD, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetOffloadPolicies(null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -416,7 +421,8 @@ public void getMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse async @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_UNACKED, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxUnackedMessagesOnConsumer(applied, isGlobal)) .thenApply(asyncResponse::resume).exceptionally(ex -> { handleTopicPolicyException("getMaxUnackedMessagesOnConsumer", ex, asyncResponse); @@ -440,7 +446,8 @@ public void setMaxUnackedMessagesOnConsumer( @ApiParam(value = "Max unacked messages on consumer policies for the specified topic") Integer maxUnackedNum) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_UNACKED, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxUnackedMessagesOnConsumer(maxUnackedNum, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -462,7 +469,8 @@ public void deleteMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse as @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_UNACKED, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxUnackedMessagesOnConsumer(null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -485,7 +493,8 @@ public void getDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyn @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName, isGlobal)) .thenAccept(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); @@ -513,7 +522,8 @@ public void setDeduplicationSnapshotInterval( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetDeduplicationSnapshotInterval(interval, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -535,7 +545,8 @@ public void deleteDeduplicationSnapshotInterval(@Suspended final AsyncResponse a @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetDeduplicationSnapshotInterval(null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -559,7 +570,8 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.INACTIVE_TOPIC, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetInactiveTopicPolicies(applied, isGlobal)) .thenApply(asyncResponse::resume).exceptionally(ex -> { handleTopicPolicyException("getInactiveTopicPolicies", ex, asyncResponse); @@ -582,7 +594,8 @@ public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @ApiParam(value = "inactive topic policies for the specified topic") InactiveTopicPolicies inactiveTopicPolicies) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetInactiveTopicPolicies(inactiveTopicPolicies, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -604,7 +617,8 @@ public void deleteInactiveTopicPolicies(@Suspended final AsyncResponse asyncResp @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetInactiveTopicPolicies(null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -628,7 +642,8 @@ public void getMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse a @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_UNACKED, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxUnackedMessagesOnSubscription(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -703,7 +718,8 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DELAYED_DELIVERY, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetDelayedDeliveryPolicies(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -1669,6 +1685,7 @@ public Response examineMessage( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); + validateTopicOperation(topicName, TopicOperation.PEEK_MESSAGES); return internalExamineMessage(initialPosition, messagePosition, authoritative); } @@ -1774,6 +1791,7 @@ public PersistentOfflineTopicStats getBacklog( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); + validateTopicOperation(topicName, TopicOperation.GET_BACKLOG_SIZE); return internalGetBacklog(authoritative); } @@ -1817,7 +1835,8 @@ public void getBacklogQuotaMap( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetBacklogQuota(applied, isGlobal)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { @@ -1895,7 +1914,8 @@ public void getReplicationClusters(@Suspended final AsyncResponse asyncResponse, + "For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) .thenAccept(op -> { asyncResponse.resume(op.map(TopicPolicies::getReplicationClustersSet).orElseGet(() -> { @@ -1977,7 +1997,8 @@ public void getMessageTTL(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.TTL, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName, isGlobal)) .thenAccept(op -> asyncResponse.resume(op .map(TopicPolicies::getMessageTTLInSeconds) @@ -2014,7 +2035,8 @@ public void setMessageTTL(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.TTL, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMessageTTL(messageTTL, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2041,7 +2063,8 @@ public void removeMessageTTL(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.TTL, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMessageTTL(null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2067,7 +2090,8 @@ public void getDeduplication(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DEDUPLICATION, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetDeduplication(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -2094,7 +2118,8 @@ public void setDeduplication( @ApiParam(value = "DeduplicationEnabled policies for the specified topic") Boolean enabled) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DEDUPLICATION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetDeduplication(enabled, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2119,7 +2144,8 @@ public void removeDeduplication(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.DEDUPLICATION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetDeduplication(null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2243,7 +2269,8 @@ public void getPersistence(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.PERSISTENCE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetPersistence(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -2271,7 +2298,8 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Bookkeeper persistence policies for specified topic") PersistencePolicies persistencePolicies) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.PERSISTENCE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetPersistence(persistencePolicies, isGlobal)) .thenRun(() -> { try { @@ -2307,7 +2335,8 @@ public void removePersistence(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.PERSISTENCE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemovePersistence(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove persistence policies: namespace={}, topic={}", @@ -2338,7 +2367,8 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxSubscriptionsPerTopic(isGlobal)) .thenAccept(op -> asyncResponse.resume(op.isPresent() ? op.get() : Response.noContent().build())) @@ -2366,7 +2396,8 @@ public void setMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "The max subscriptions of the topic") int maxSubscriptionsPerTopic) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxSubscriptionsPerTopic(maxSubscriptionsPerTopic, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully updated maxSubscriptionsPerTopic: namespace={}, topic={}" @@ -2396,7 +2427,8 @@ public void removeMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncR @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxSubscriptionsPerTopic(null, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove maxSubscriptionsPerTopic: namespace={}, topic={}", @@ -2426,7 +2458,8 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION_RATE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetReplicatorDispatchRate(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -2453,7 +2486,8 @@ public void setReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Replicator dispatch rate of the topic") DispatchRateImpl dispatchRate) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetReplicatorDispatchRate(dispatchRate, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully updated replicatorDispatchRate: namespace={}, topic={}" @@ -2483,7 +2517,8 @@ public void removeReplicatorDispatchRate(@Suspended final AsyncResponse asyncRes @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetReplicatorDispatchRate(null, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove replicatorDispatchRate limit: namespace={}, topic={}", @@ -2513,7 +2548,8 @@ public void getMaxProducers(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_PRODUCERS, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxProducers(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -2540,7 +2576,8 @@ public void setMaxProducers(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @ApiParam(value = "The max producers of the topic") int maxProducers) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxProducers(maxProducers, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully updated max producers: namespace={}, topic={}, maxProducers={}", @@ -2572,7 +2609,8 @@ public void removeMaxProducers(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveMaxProducers(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove max producers: namespace={}, topic={}", @@ -2604,7 +2642,8 @@ public void getMaxConsumers(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_CONSUMERS, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxConsumers(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -2631,7 +2670,8 @@ public void setMaxConsumers(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "The max consumers of the topic") int maxConsumers) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxConsumers(maxConsumers, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully updated max consumers: namespace={}, topic={}, maxConsumers={}", @@ -2663,7 +2703,8 @@ public void removeMaxConsumers(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveMaxConsumers(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove max consumers: namespace={}, topic={}", @@ -2694,7 +2735,8 @@ public void getMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateAdminAccessForTenantAsync(topicName.getTenant()) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxMessageSize(isGlobal)) .thenAccept(policies -> { asyncResponse.resume(policies.isPresent() ? policies.get() : Response.noContent().build()); @@ -2723,7 +2765,8 @@ public void setMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "The max message size of the topic") int maxMessageSize) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateAdminAccessForTenantAsync(topicName.getTenant()) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxMessageSize(maxMessageSize, isGlobal)) .thenRun(() -> { log.info( @@ -2757,7 +2800,8 @@ public void removeMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateAdminAccessForTenantAsync(topicName.getTenant()) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxMessageSize(null, isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove max message size: namespace={}, topic={}", @@ -3008,7 +3052,8 @@ public void getDispatchRate(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetDispatchRate(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -3034,7 +3079,8 @@ public void setDispatchRate(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @ApiParam(value = "Dispatch rate for the specified topic") DispatchRateImpl dispatchRate) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetDispatchRate(dispatchRate, isGlobal)) .thenRun(() -> { try { @@ -3070,7 +3116,8 @@ public void removeDispatchRate(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveDispatchRate(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove topic dispatch rate: tenant={}, namespace={}, topic={}", @@ -3103,7 +3150,8 @@ public void getSubscriptionDispatchRate(@Suspended final AsyncResponse asyncResp @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetSubscriptionDispatchRate(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -3131,7 +3179,8 @@ public void setSubscriptionDispatchRate( @ApiParam(value = "Subscription message dispatch rate for the specified topic") DispatchRateImpl dispatchRate) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSubscriptionDispatchRate(dispatchRate, isGlobal)) .thenRun(() -> { try { @@ -3167,7 +3216,8 @@ public void removeSubscriptionDispatchRate(@Suspended final AsyncResponse asyncR @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveSubscriptionDispatchRate(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove topic subscription dispatch rate: tenant={}, namespace={}, topic={}", @@ -3200,7 +3250,8 @@ public void getCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.COMPACTION, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetCompactionThreshold(applied, isGlobal)) .thenApply(asyncResponse::resume) .exceptionally(ex -> { @@ -3226,7 +3277,8 @@ public void setCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @ApiParam(value = "Dispatch rate for the specified topic") long compactionThreshold) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.COMPACTION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetCompactionThreshold(compactionThreshold, isGlobal)) .thenRun(() -> { try { @@ -3262,7 +3314,8 @@ public void removeCompactionThreshold(@Suspended final AsyncResponse asyncRespon @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.COMPACTION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveCompactionThreshold(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove topic compaction threshold: tenant={}, namespace={}, topic={}", @@ -3294,7 +3347,8 @@ public void getMaxConsumersPerSubscription(@Suspended final AsyncResponse asyncR @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_CONSUMERS, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetMaxConsumersPerSubscription(isGlobal)) .thenAccept(op -> asyncResponse.resume(op.isPresent() ? op.get() : Response.noContent().build())) @@ -3322,7 +3376,8 @@ public void setMaxConsumersPerSubscription( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Dispatch rate for the specified topic") int maxConsumersPerSubscription) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetMaxConsumersPerSubscription(maxConsumersPerSubscription, isGlobal)) .thenRun(() -> { try { @@ -3358,7 +3413,8 @@ public void removeMaxConsumersPerSubscription(@Suspended final AsyncResponse asy @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveMaxConsumersPerSubscription(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove topic max consumers per subscription:" @@ -3391,7 +3447,8 @@ public void getPublishRate(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetPublishRate(isGlobal)) .thenAccept(op -> asyncResponse.resume(op.isPresent() ? op.get() : Response.noContent().build())) @@ -3418,7 +3475,8 @@ public void setPublishRate(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Dispatch rate for the specified topic") PublishRate publishRate) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetPublishRate(publishRate, isGlobal)) .thenRun(() -> { try { @@ -3455,7 +3513,8 @@ public void removePublishRate(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemovePublishRate(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove topic publish rate: tenant={}, namespace={}, topic={}, isGlobal={}", @@ -3488,7 +3547,8 @@ public void getSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetSubscriptionTypesEnabled(isGlobal)) .thenAccept(op -> { asyncResponse.resume(op.isPresent() ? op.get() @@ -3518,7 +3578,8 @@ public void setSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @ApiParam(value = "Enable sub types for the specified topic") Set subscriptionTypesEnabled) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSubscriptionTypesEnabled(subscriptionTypesEnabled, isGlobal)) .thenRun(() -> { try { @@ -3554,7 +3615,8 @@ public void removeSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncR @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveSubscriptionTypesEnabled(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove subscription types enabled: namespace={}, topic={}", @@ -3586,7 +3648,8 @@ public void getSubscribeRate(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetSubscribeRate(applied, isGlobal)) .thenApply(asyncResponse::resume).exceptionally(ex -> { handleTopicPolicyException("getSubscribeRate", ex, asyncResponse); @@ -3612,7 +3675,8 @@ public void setSubscribeRate( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Subscribe rate for the specified topic") SubscribeRate subscribeRate) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSubscribeRate(subscribeRate, isGlobal)) .thenRun(() -> { try { @@ -3650,7 +3714,8 @@ public void removeSubscribeRate(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Subscribe rate for the specified topic") SubscribeRate subscribeRate) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RATE, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveSubscribeRate(isGlobal)) .thenRun(() -> { log.info( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java index 2a2f329d5f47e..d63da5b7718aa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java @@ -38,6 +38,8 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -46,7 +48,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; - public final class TopicPoliciesAuthZTest extends MockedPulsarServiceBaseTest { private PulsarAdmin superUserAdmin; @@ -219,4 +220,207 @@ public void testRetention() { superUserAdmin.namespaces().revokePermissionsOnNamespace("public/default", subject); } } + + @SneakyThrows + @Test + public void testMaxUnackedMessagesOnConsumer() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // mocked data + int definedUnackedMessagesOnConsumer = 100; + + // test superuser + superUserAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, definedUnackedMessagesOnConsumer); + + // because the topic policies is eventual consistency, we should wait here + await().untilAsserted(() -> { + final int unackedMessagesOnConsumer = superUserAdmin.topicPolicies() + .getMaxUnackedMessagesOnConsumer(topic); + Assert.assertEquals(unackedMessagesOnConsumer, definedUnackedMessagesOnConsumer); + }); + superUserAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic); + + await().untilAsserted(() -> { + final Integer unackedMessagesOnConsumer = superUserAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic); + Assert.assertNull(unackedMessagesOnConsumer); + }); + + // test tenant manager + + tenantManagerAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, definedUnackedMessagesOnConsumer); + await().untilAsserted(() -> { + final int unackedMessagesOnConsumer = tenantManagerAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic); + Assert.assertEquals(unackedMessagesOnConsumer, definedUnackedMessagesOnConsumer); + }); + tenantManagerAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic); + await().untilAsserted(() -> { + final Integer unackedMessagesOnConsumer = tenantManagerAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic); + Assert.assertNull(unackedMessagesOnConsumer); + }); + + // test nobody + + try { + subAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, definedUnackedMessagesOnConsumer); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + // test sub user with permissions + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", + subject, Sets.newHashSet(action)); + try { + subAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, definedUnackedMessagesOnConsumer); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace("public/default", subject); + } + } + + @SneakyThrows + @Test + public void testMaxUnackedMessagesOnSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // mocked data + int definedUnackedMessagesOnConsumer = 100; + + // test superuser + superUserAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, definedUnackedMessagesOnConsumer); + + // because the topic policies is eventual consistency, we should wait here + await().untilAsserted(() -> { + final int unackedMessagesOnConsumer = superUserAdmin.topicPolicies() + .getMaxUnackedMessagesOnSubscription(topic); + Assert.assertEquals(unackedMessagesOnConsumer, definedUnackedMessagesOnConsumer); + }); + superUserAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic); + + await().untilAsserted(() -> { + final Integer unackedMessagesOnConsumer = superUserAdmin.topicPolicies() + .getMaxUnackedMessagesOnSubscription(topic); + Assert.assertNull(unackedMessagesOnConsumer); + }); + + // test tenant manager + + tenantManagerAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, definedUnackedMessagesOnConsumer); + await().untilAsserted(() -> { + final int unackedMessagesOnConsumer = tenantManagerAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic); + Assert.assertEquals(unackedMessagesOnConsumer, definedUnackedMessagesOnConsumer); + }); + tenantManagerAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic); + await().untilAsserted(() -> { + final Integer unackedMessagesOnConsumer = tenantManagerAdmin.topicPolicies() + .getMaxUnackedMessagesOnSubscription(topic); + Assert.assertNull(unackedMessagesOnConsumer); + }); + + // test nobody + + try { + subAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, definedUnackedMessagesOnConsumer); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + // test sub user with permissions + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", + subject, Sets.newHashSet(action)); + try { + subAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, definedUnackedMessagesOnConsumer); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace("public/default", subject); + } + + } } From a76ddbe5af523b4aa541a2272c58f685ef05859f Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Thu, 7 Mar 2024 14:15:15 +0800 Subject: [PATCH 08/16] [fix][build] Delete unused import --- .../org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java index d63da5b7718aa..b76c6e3f3f2ec 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java @@ -38,8 +38,6 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.OffloadPolicies; -import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.ObjectMapperFactory; From 5812b306e6d89ba187dc35a008dfb2a63c522efa Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Fri, 8 Mar 2024 17:42:27 +0800 Subject: [PATCH 09/16] Bump version to 2.10.0-SNAPSHOT --- bouncy-castle/bc/pom.xml | 2 +- bouncy-castle/bcfips-include-test/pom.xml | 2 +- bouncy-castle/bcfips/pom.xml | 2 +- bouncy-castle/pom.xml | 2 +- buildtools/pom.xml | 2 +- distribution/io/pom.xml | 2 +- distribution/offloaders/pom.xml | 2 +- distribution/pom.xml | 2 +- distribution/server/pom.xml | 2 +- docker/pom.xml | 2 +- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- kafka-connect-avro-converter-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- pom.xml | 2 +- pulsar-broker-auth-athenz/pom.xml | 2 +- pulsar-broker-auth-sasl/pom.xml | 2 +- pulsar-broker-common/pom.xml | 2 +- pulsar-broker-shaded/pom.xml | 2 +- pulsar-broker/pom.xml | 2 +- pulsar-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml | 2 +- pulsar-client-admin-api/pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 2 +- pulsar-client-admin/pom.xml | 2 +- pulsar-client-all/pom.xml | 2 +- pulsar-client-api/pom.xml | 2 +- pulsar-client-auth-athenz/pom.xml | 2 +- pulsar-client-auth-sasl/pom.xml | 2 +- pulsar-client-messagecrypto-bc/pom.xml | 2 +- pulsar-client-shaded/pom.xml | 2 +- pulsar-client-tools-test/pom.xml | 2 +- pulsar-client-tools/pom.xml | 2 +- pulsar-client/pom.xml | 2 +- pulsar-common/pom.xml | 2 +- pulsar-config-validation/pom.xml | 2 +- pulsar-functions/api-java/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/java-examples/pom.xml | 2 +- pulsar-functions/localrun-shaded/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/pom.xml | 2 +- pulsar-functions/proto/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-functions/runtime/pom.xml | 2 +- pulsar-functions/secrets/pom.xml | 2 +- pulsar-functions/utils/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- pulsar-io/aerospike/pom.xml | 2 +- pulsar-io/aws/pom.xml | 2 +- pulsar-io/batch-data-generator/pom.xml | 2 +- pulsar-io/batch-discovery-triggerers/pom.xml | 2 +- pulsar-io/canal/pom.xml | 2 +- pulsar-io/cassandra/pom.xml | 2 +- pulsar-io/common/pom.xml | 2 +- pulsar-io/core/pom.xml | 2 +- pulsar-io/data-generator/pom.xml | 2 +- pulsar-io/debezium/core/pom.xml | 2 +- pulsar-io/debezium/mongodb/pom.xml | 2 +- pulsar-io/debezium/mssql/pom.xml | 2 +- pulsar-io/debezium/mysql/pom.xml | 2 +- pulsar-io/debezium/oracle/pom.xml | 2 +- pulsar-io/debezium/pom.xml | 2 +- pulsar-io/debezium/postgres/pom.xml | 2 +- pulsar-io/docs/pom.xml | 2 +- pulsar-io/dynamodb/pom.xml | 2 +- pulsar-io/elastic-search/pom.xml | 2 +- pulsar-io/file/pom.xml | 2 +- pulsar-io/flume/pom.xml | 2 +- pulsar-io/hbase/pom.xml | 2 +- pulsar-io/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/influxdb/pom.xml | 2 +- pulsar-io/jdbc/clickhouse/pom.xml | 2 +- pulsar-io/jdbc/core/pom.xml | 2 +- pulsar-io/jdbc/mariadb/pom.xml | 2 +- pulsar-io/jdbc/pom.xml | 2 +- pulsar-io/jdbc/postgres/pom.xml | 2 +- pulsar-io/jdbc/sqlite/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor-nar/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor/pom.xml | 2 +- pulsar-io/kafka/pom.xml | 2 +- pulsar-io/kinesis/pom.xml | 2 +- pulsar-io/mongo/pom.xml | 2 +- pulsar-io/netty/pom.xml | 2 +- pulsar-io/nsq/pom.xml | 2 +- pulsar-io/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 2 +- pulsar-io/redis/pom.xml | 2 +- pulsar-io/solr/pom.xml | 2 +- pulsar-io/twitter/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-package-management/bookkeeper-storage/pom.xml | 2 +- pulsar-package-management/core/pom.xml | 2 +- pulsar-package-management/filesystem-storage/pom.xml | 2 +- pulsar-package-management/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- pulsar-sql/java-version-trim-agent/pom.xml | 2 +- pulsar-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/pom.xml | 2 +- pulsar-testclient/pom.xml | 2 +- pulsar-transaction/common/pom.xml | 2 +- pulsar-transaction/coordinator/pom.xml | 2 +- pulsar-transaction/pom.xml | 2 +- pulsar-websocket/pom.xml | 2 +- structured-event-log/pom.xml | 2 +- testmocks/pom.xml | 2 +- tests/bc_2_0_0/pom.xml | 2 +- tests/bc_2_0_1/pom.xml | 2 +- tests/bc_2_6_0/pom.xml | 2 +- tests/docker-images/java-test-functions/pom.xml | 2 +- tests/docker-images/java-test-image/pom.xml | 2 +- tests/docker-images/latest-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/pom.xml | 2 +- tests/pulsar-client-admin-shade-test/pom.xml | 2 +- tests/pulsar-client-all-shade-test/pom.xml | 2 +- tests/pulsar-client-shade-test/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 2 +- tiered-storage/jcloud/pom.xml | 2 +- tiered-storage/pom.xml | 2 +- 126 files changed, 126 insertions(+), 126 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 1b5c28a6a7fd0..f788b1096fcd3 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 578e7b56c9a81..0cc21ada0949a 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar bouncy-castle-parent - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 85f34a4556909..e52c44a94f209 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index da23d3775e7e3..4942876901f70 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 37130271def02..391de87aae855 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,7 +31,7 @@ org.apache.pulsar buildtools - 2.10.6 + 2.10.7-SNAPSHOT jar Pulsar Build Tools diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 95cbeb9b510aa..7b50568ecbd2d 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 4c539ee71161f..72018f112219f 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 69521c814e77b..85babe90480d6 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index dca89684cf025..35fcea819ff14 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/docker/pom.xml b/docker/pom.xml index 0a5f85f6c5f8e..0b340ca4653b2 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 318815f2eac14..702fdd2782c4e 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 24259a73aeb06..a5f489a04f662 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 0160ae0ce66e5..f2ac79516a7bd 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/kafka-connect-avro-converter-shaded/pom.xml b/kafka-connect-avro-converter-shaded/pom.xml index f38bc7d0c4187..b0f53aeff8aa3 100644 --- a/kafka-connect-avro-converter-shaded/pom.xml +++ b/kafka-connect-avro-converter-shaded/pom.xml @@ -26,7 +26,7 @@ pulsar org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index d3411be3ad975..1fc12fffc1d71 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 1a12ff8109d56..65f11ba8776bf 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very diff --git a/pulsar-broker-auth-athenz/pom.xml b/pulsar-broker-auth-athenz/pom.xml index 2b3540f7ee581..398b3fee8d052 100644 --- a/pulsar-broker-auth-athenz/pom.xml +++ b/pulsar-broker-auth-athenz/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-broker-auth-athenz diff --git a/pulsar-broker-auth-sasl/pom.xml b/pulsar-broker-auth-sasl/pom.xml index 6cb359b07446b..7b4c84d4b31a8 100644 --- a/pulsar-broker-auth-sasl/pom.xml +++ b/pulsar-broker-auth-sasl/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-broker-auth-sasl diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index e1e3d0d87bc04..66230647315d9 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-broker-common diff --git a/pulsar-broker-shaded/pom.xml b/pulsar-broker-shaded/pom.xml index 53a7db3cf6057..873b7e392ce3f 100644 --- a/pulsar-broker-shaded/pom.xml +++ b/pulsar-broker-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index e33f5a683758b..641e9272c294f 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-1x-base/pom.xml b/pulsar-client-1x-base/pom.xml index a795409bc34a6..09681ab56a2f2 100644 --- a/pulsar-client-1x-base/pom.xml +++ b/pulsar-client-1x-base/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-1x-base/pulsar-client-1x/pom.xml b/pulsar-client-1x-base/pulsar-client-1x/pom.xml index 0f1b7caef79ae..1f496015e8c19 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-1x/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-client-1x-base - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml index 6ab11f3a39cc4..b39f7fc63527c 100644 --- a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-client-1x-base - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-admin-api/pom.xml b/pulsar-client-admin-api/pom.xml index 326a6e64adab6..66fc9943c980f 100644 --- a/pulsar-client-admin-api/pom.xml +++ b/pulsar-client-admin-api/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 3d45d3047eeb7..2121f4edd1f25 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 51ea727388307..e0707ffc7e5f7 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 0bd7e822fa668..fbd1b2d5cc9c4 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-api/pom.xml b/pulsar-client-api/pom.xml index 12700ceba27cc..17807b5344c5b 100644 --- a/pulsar-client-api/pom.xml +++ b/pulsar-client-api/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-auth-athenz/pom.xml b/pulsar-client-auth-athenz/pom.xml index 523dc8c23f6cf..947016bff3473 100644 --- a/pulsar-client-auth-athenz/pom.xml +++ b/pulsar-client-auth-athenz/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-auth-sasl/pom.xml b/pulsar-client-auth-sasl/pom.xml index c78e06caece61..89a2fee07cdb1 100644 --- a/pulsar-client-auth-sasl/pom.xml +++ b/pulsar-client-auth-sasl/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-messagecrypto-bc/pom.xml b/pulsar-client-messagecrypto-bc/pom.xml index 7f87fe122cd31..c273fc6ceaf8e 100644 --- a/pulsar-client-messagecrypto-bc/pom.xml +++ b/pulsar-client-messagecrypto-bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 689733d659447..9f01e96d6ee60 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index c723143917fee..7591fb12d82be 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 068f8c744bf54..98d4e529a18aa 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 9b19613835d5d..13987177b7ad4 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 2f3e48ce2be85..ab3899fd665d7 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-config-validation/pom.xml b/pulsar-config-validation/pom.xml index c02d88d4aa1af..17cef90f4e2ad 100644 --- a/pulsar-config-validation/pom.xml +++ b/pulsar-config-validation/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-functions/api-java/pom.xml b/pulsar-functions/api-java/pom.xml index a9406921b1b7f..d08af8aa2f911 100644 --- a/pulsar-functions/api-java/pom.xml +++ b/pulsar-functions/api-java/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-api diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index ee8c5374bb182..6bd56ad1cc545 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-instance diff --git a/pulsar-functions/java-examples/pom.xml b/pulsar-functions/java-examples/pom.xml index 98cd1114db58f..e688d5d9c36cb 100644 --- a/pulsar-functions/java-examples/pom.xml +++ b/pulsar-functions/java-examples/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-api-examples diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index baf2b72e00178..2fe81aa49e326 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index 81a2550d6f196..e607d9e69d375 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-functions/pom.xml b/pulsar-functions/pom.xml index 895ab9bef46d8..bb6e49fec3dd9 100644 --- a/pulsar-functions/pom.xml +++ b/pulsar-functions/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions diff --git a/pulsar-functions/proto/pom.xml b/pulsar-functions/proto/pom.xml index aeeb39a5c6183..5b9a2507069dc 100644 --- a/pulsar-functions/proto/pom.xml +++ b/pulsar-functions/proto/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-proto diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 1045fadd29e5b..277f5fac7b544 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index 0154e62233eac..3def6907d6366 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-runtime diff --git a/pulsar-functions/secrets/pom.xml b/pulsar-functions/secrets/pom.xml index 4c2c201165dce..31e7aa8e4ec25 100644 --- a/pulsar-functions/secrets/pom.xml +++ b/pulsar-functions/secrets/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-secrets diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 2a0d035a6cf0f..39d8e91ed489b 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT pulsar-functions-utils diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index f83b6b17f2c12..b9f0e156eb697 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-functions - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-io/aerospike/pom.xml b/pulsar-io/aerospike/pom.xml index 771c909507b43..5672e064fb2e9 100644 --- a/pulsar-io/aerospike/pom.xml +++ b/pulsar-io/aerospike/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-aerospike diff --git a/pulsar-io/aws/pom.xml b/pulsar-io/aws/pom.xml index d2c42408cc2d2..9f3aa26998c82 100644 --- a/pulsar-io/aws/pom.xml +++ b/pulsar-io/aws/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-aws diff --git a/pulsar-io/batch-data-generator/pom.xml b/pulsar-io/batch-data-generator/pom.xml index bb4fa445bbca3..f9a6e54c2f093 100644 --- a/pulsar-io/batch-data-generator/pom.xml +++ b/pulsar-io/batch-data-generator/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-batch-data-generator diff --git a/pulsar-io/batch-discovery-triggerers/pom.xml b/pulsar-io/batch-discovery-triggerers/pom.xml index 82672d464cd58..8ce0f8d540a27 100644 --- a/pulsar-io/batch-discovery-triggerers/pom.xml +++ b/pulsar-io/batch-discovery-triggerers/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-batch-discovery-triggerers diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index bb41b142cf6c6..f026aaec88ebc 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-io/cassandra/pom.xml b/pulsar-io/cassandra/pom.xml index 7a8d5795bf790..78d726ab0aa81 100644 --- a/pulsar-io/cassandra/pom.xml +++ b/pulsar-io/cassandra/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-cassandra diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index 3c9bb46d20231..934580e622d1d 100644 --- a/pulsar-io/common/pom.xml +++ b/pulsar-io/common/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-common diff --git a/pulsar-io/core/pom.xml b/pulsar-io/core/pom.xml index d341e110927d0..73240e1e356af 100644 --- a/pulsar-io/core/pom.xml +++ b/pulsar-io/core/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-core diff --git a/pulsar-io/data-generator/pom.xml b/pulsar-io/data-generator/pom.xml index 6a95d07e1b54d..0753ad00036f2 100644 --- a/pulsar-io/data-generator/pom.xml +++ b/pulsar-io/data-generator/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-data-generator diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index 46b7cdd592d80..62e8a4e2d2b95 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium-core diff --git a/pulsar-io/debezium/mongodb/pom.xml b/pulsar-io/debezium/mongodb/pom.xml index 0a01f218a41f3..b536cc77f63e2 100644 --- a/pulsar-io/debezium/mongodb/pom.xml +++ b/pulsar-io/debezium/mongodb/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium-mongodb diff --git a/pulsar-io/debezium/mssql/pom.xml b/pulsar-io/debezium/mssql/pom.xml index c076715fbc787..1134d88759a2e 100644 --- a/pulsar-io/debezium/mssql/pom.xml +++ b/pulsar-io/debezium/mssql/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium-mssql diff --git a/pulsar-io/debezium/mysql/pom.xml b/pulsar-io/debezium/mysql/pom.xml index 03e396cfaa8f5..fc0b1900da5c6 100644 --- a/pulsar-io/debezium/mysql/pom.xml +++ b/pulsar-io/debezium/mysql/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium-mysql diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index d7fefcd5a0d6b..7f8179e799fe1 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium-oracle diff --git a/pulsar-io/debezium/pom.xml b/pulsar-io/debezium/pom.xml index aa024b86b6521..18c9a572a8d51 100644 --- a/pulsar-io/debezium/pom.xml +++ b/pulsar-io/debezium/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium diff --git a/pulsar-io/debezium/postgres/pom.xml b/pulsar-io/debezium/postgres/pom.xml index 1645bdf1cf135..3d3dd4769f01f 100644 --- a/pulsar-io/debezium/postgres/pom.xml +++ b/pulsar-io/debezium/postgres/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io-debezium - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-debezium-postgres diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index 13f5a001440a5..dbac418b50411 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-docs diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index 34c7f7e2961f1..4f6aa52241d53 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-dynamodb diff --git a/pulsar-io/elastic-search/pom.xml b/pulsar-io/elastic-search/pom.xml index 4b00d709f1825..16c093e4c6009 100644 --- a/pulsar-io/elastic-search/pom.xml +++ b/pulsar-io/elastic-search/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-elastic-search Pulsar IO :: ElasticSearch diff --git a/pulsar-io/file/pom.xml b/pulsar-io/file/pom.xml index 1b78f1bc1988b..8f52d410f9240 100644 --- a/pulsar-io/file/pom.xml +++ b/pulsar-io/file/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-file diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index 7e62c299f01cc..ebb4f61ee2d62 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-flume diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index 6e9a7dc2e1418..8be226dbf0da5 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-hbase Pulsar IO :: Hbase diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index 29a52b17ff551..4639e048784a6 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-hdfs2 Pulsar IO :: Hdfs2 diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 68327c7166a2a..63180bcd43b19 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-hdfs3 Pulsar IO :: Hdfs3 diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index 4a9ea205f6c00..b3a9483e88cca 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-influxdb diff --git a/pulsar-io/jdbc/clickhouse/pom.xml b/pulsar-io/jdbc/clickhouse/pom.xml index 0745d9950a76c..947a75b7a17fd 100644 --- a/pulsar-io/jdbc/clickhouse/pom.xml +++ b/pulsar-io/jdbc/clickhouse/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index c2170c174e918..9e0a641dd3c6b 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-io/jdbc/mariadb/pom.xml b/pulsar-io/jdbc/mariadb/pom.xml index d909af3bf165c..b051d9c3e1632 100644 --- a/pulsar-io/jdbc/mariadb/pom.xml +++ b/pulsar-io/jdbc/mariadb/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-io/jdbc/pom.xml b/pulsar-io/jdbc/pom.xml index 85e71ddd6c683..0e7a0a7523656 100644 --- a/pulsar-io/jdbc/pom.xml +++ b/pulsar-io/jdbc/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-jdbc diff --git a/pulsar-io/jdbc/postgres/pom.xml b/pulsar-io/jdbc/postgres/pom.xml index f9fe74a07ebfb..dfe7faf799627 100644 --- a/pulsar-io/jdbc/postgres/pom.xml +++ b/pulsar-io/jdbc/postgres/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-io/jdbc/sqlite/pom.xml b/pulsar-io/jdbc/sqlite/pom.xml index e5fca91b6c582..2d28bcfdd35b5 100644 --- a/pulsar-io/jdbc/sqlite/pom.xml +++ b/pulsar-io/jdbc/sqlite/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 pulsar-io-jdbc-sqlite diff --git a/pulsar-io/kafka-connect-adaptor-nar/pom.xml b/pulsar-io/kafka-connect-adaptor-nar/pom.xml index 76d532dcdd0fd..c985a20aa0e5f 100644 --- a/pulsar-io/kafka-connect-adaptor-nar/pom.xml +++ b/pulsar-io/kafka-connect-adaptor-nar/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-kafka-connect-adaptor-nar diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 94b3ac3a72c65..7290b9e4e6b2b 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-kafka-connect-adaptor diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 19c6518df68b2..c8433ec52007f 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-kafka diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index 74d0e57aedaf3..a36f0f7a386cd 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-kinesis diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index 5024127718bb3..c5c1ffa65134d 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-mongo diff --git a/pulsar-io/netty/pom.xml b/pulsar-io/netty/pom.xml index 40912b136225b..7fad90f76ed84 100644 --- a/pulsar-io/netty/pom.xml +++ b/pulsar-io/netty/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-netty diff --git a/pulsar-io/nsq/pom.xml b/pulsar-io/nsq/pom.xml index 2a41f92dac4a4..a69073f0e0da6 100644 --- a/pulsar-io/nsq/pom.xml +++ b/pulsar-io/nsq/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-nsq diff --git a/pulsar-io/pom.xml b/pulsar-io/pom.xml index f0dc1551f9a9e..693930fa35599 100644 --- a/pulsar-io/pom.xml +++ b/pulsar-io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 8397fc84b361a..05dfeec1596d0 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-rabbitmq diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 64ba40fb3e685..03d2f27e5db6f 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-redis diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index ee5072ce2faec..513aa0fbc92db 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -25,7 +25,7 @@ pulsar-io org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT diff --git a/pulsar-io/twitter/pom.xml b/pulsar-io/twitter/pom.xml index 9167b3a28f97e..cf380199d9fc2 100644 --- a/pulsar-io/twitter/pom.xml +++ b/pulsar-io/twitter/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar-io - 2.10.6 + 2.10.7-SNAPSHOT pulsar-io-twitter diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 3c5e0eb40ecd1..16f6751be0d45 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-package-management/bookkeeper-storage/pom.xml b/pulsar-package-management/bookkeeper-storage/pom.xml index 1af8f528816bc..6cb00347948f8 100644 --- a/pulsar-package-management/bookkeeper-storage/pom.xml +++ b/pulsar-package-management/bookkeeper-storage/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-package-management/core/pom.xml b/pulsar-package-management/core/pom.xml index 749ec9313cb3d..52761d53b5e34 100644 --- a/pulsar-package-management/core/pom.xml +++ b/pulsar-package-management/core/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-package-management/filesystem-storage/pom.xml b/pulsar-package-management/filesystem-storage/pom.xml index 3958ce4c313b2..4bc5e457747ef 100644 --- a/pulsar-package-management/filesystem-storage/pom.xml +++ b/pulsar-package-management/filesystem-storage/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-package-management/pom.xml b/pulsar-package-management/pom.xml index 5f91f6eb84ef9..1051075180166 100644 --- a/pulsar-package-management/pom.xml +++ b/pulsar-package-management/pom.xml @@ -25,7 +25,7 @@ pulsar org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. 4.0.0 diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index cd408cde17020..6c58528668bd4 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-proxy diff --git a/pulsar-sql/java-version-trim-agent/pom.xml b/pulsar-sql/java-version-trim-agent/pom.xml index cb2e6138c1ded..38948a5b44888 100644 --- a/pulsar-sql/java-version-trim-agent/pom.xml +++ b/pulsar-sql/java-version-trim-agent/pom.xml @@ -24,7 +24,7 @@ pulsar-sql org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/pulsar-sql/pom.xml b/pulsar-sql/pom.xml index 69d313469939b..cddc8f4709669 100644 --- a/pulsar-sql/pom.xml +++ b/pulsar-sql/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-sql diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index a45c4cf080682..a5905c5b22620 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-sql - 2.10.6 + 2.10.7-SNAPSHOT pulsar-presto-distribution diff --git a/pulsar-sql/presto-pulsar-plugin/pom.xml b/pulsar-sql/presto-pulsar-plugin/pom.xml index 6ec62caec43f8..6da691736a38d 100644 --- a/pulsar-sql/presto-pulsar-plugin/pom.xml +++ b/pulsar-sql/presto-pulsar-plugin/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-sql - 2.10.6 + 2.10.7-SNAPSHOT pulsar-presto-connector diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index bb0e06d3f2e08..5b2f0fdd74985 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar-sql - 2.10.6 + 2.10.7-SNAPSHOT pulsar-presto-connector-original diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 6433f9db5a176..a8a9a75da4f0b 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/pulsar-transaction/common/pom.xml b/pulsar-transaction/common/pom.xml index 25a01608f2ce5..5af0a229a5a0d 100644 --- a/pulsar-transaction/common/pom.xml +++ b/pulsar-transaction/common/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-transaction-parent - 2.10.6 + 2.10.7-SNAPSHOT pulsar-transaction-common diff --git a/pulsar-transaction/coordinator/pom.xml b/pulsar-transaction/coordinator/pom.xml index 7b5d45bf1e83a..13b4417ff7297 100644 --- a/pulsar-transaction/coordinator/pom.xml +++ b/pulsar-transaction/coordinator/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar pulsar-transaction-parent - 2.10.6 + 2.10.7-SNAPSHOT pulsar-transaction-coordinator diff --git a/pulsar-transaction/pom.xml b/pulsar-transaction/pom.xml index 23ed756b1ffa2..e36e5890ecb08 100644 --- a/pulsar-transaction/pom.xml +++ b/pulsar-transaction/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT pulsar-transaction-parent diff --git a/pulsar-websocket/pom.xml b/pulsar-websocket/pom.xml index 12282020f1fe0..13a75fc88d6f1 100644 --- a/pulsar-websocket/pom.xml +++ b/pulsar-websocket/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index ad12dc090defd..ef23e9da88758 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/testmocks/pom.xml b/testmocks/pom.xml index a9ebf86e78027..e4902e47dfeb5 100644 --- a/testmocks/pom.xml +++ b/testmocks/pom.xml @@ -25,7 +25,7 @@ pulsar org.apache.pulsar - 2.10.6 + 2.10.7-SNAPSHOT testmocks diff --git a/tests/bc_2_0_0/pom.xml b/tests/bc_2_0_0/pom.xml index 74e6b29e4b1d6..13e1e9b5be493 100644 --- a/tests/bc_2_0_0/pom.xml +++ b/tests/bc_2_0_0/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT bc_2_0_0 diff --git a/tests/bc_2_0_1/pom.xml b/tests/bc_2_0_1/pom.xml index 8064d106e302f..eff4ce3d7be75 100644 --- a/tests/bc_2_0_1/pom.xml +++ b/tests/bc_2_0_1/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT bc_2_0_1 diff --git a/tests/bc_2_6_0/pom.xml b/tests/bc_2_6_0/pom.xml index 92680683c9e0d..3ce3aa5617f8b 100644 --- a/tests/bc_2_6_0/pom.xml +++ b/tests/bc_2_6_0/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 diff --git a/tests/docker-images/java-test-functions/pom.xml b/tests/docker-images/java-test-functions/pom.xml index b385c4ab7c521..3af83db9b2f6f 100644 --- a/tests/docker-images/java-test-functions/pom.xml +++ b/tests/docker-images/java-test-functions/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 java-test-functions diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml index ee5d46d41294e..c922cfd79c026 100644 --- a/tests/docker-images/java-test-image/pom.xml +++ b/tests/docker-images/java-test-image/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 java-test-image diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index feab4c22c37f9..66f5ec51d4361 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 2.10.6 + 2.10.7-SNAPSHOT 4.0.0 latest-version-image diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml index 78d96ebd53d75..59d808a70bf9b 100644 --- a/tests/docker-images/pom.xml +++ b/tests/docker-images/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT docker-images Apache Pulsar :: Tests :: Docker Images diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index f4ee351072f05..c9da4011938ca 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT integration diff --git a/tests/pom.xml b/tests/pom.xml index aaf1eedfd1902..59f055c3b72e4 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT org.apache.pulsar.tests tests-parent diff --git a/tests/pulsar-client-admin-shade-test/pom.xml b/tests/pulsar-client-admin-shade-test/pom.xml index 828c0e554edd7..dc06568d30545 100644 --- a/tests/pulsar-client-admin-shade-test/pom.xml +++ b/tests/pulsar-client-admin-shade-test/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT pulsar-client-admin-shade-test diff --git a/tests/pulsar-client-all-shade-test/pom.xml b/tests/pulsar-client-all-shade-test/pom.xml index 9af89ea26e7d9..f0560b5733833 100644 --- a/tests/pulsar-client-all-shade-test/pom.xml +++ b/tests/pulsar-client-all-shade-test/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT pulsar-client-all-shade-test diff --git a/tests/pulsar-client-shade-test/pom.xml b/tests/pulsar-client-shade-test/pom.xml index 50d7ca86e9955..d9f74e8c374cf 100644 --- a/tests/pulsar-client-shade-test/pom.xml +++ b/tests/pulsar-client-shade-test/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar.tests tests-parent - 2.10.6 + 2.10.7-SNAPSHOT pulsar-client-shade-test diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 73ba9a72dc01e..1100be77e550b 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar tiered-storage-parent - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index e191e123b8bdf..8c03df5a8a783 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar tiered-storage-parent - 2.10.6 + 2.10.7-SNAPSHOT .. diff --git a/tiered-storage/pom.xml b/tiered-storage/pom.xml index 6e4a8dc2f2b4d..784506d176f7d 100644 --- a/tiered-storage/pom.xml +++ b/tiered-storage/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 2.10.6 + 2.10.7-SNAPSHOT .. From 2c85dd13384ea7622d9bb0715d04cb5e3dc4ed95 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 14 Mar 2024 10:24:36 +0800 Subject: [PATCH 10/16] [fix][broker] Fix incorrect unack msk count when dup ack a message (#20990) (#22223) --- .../pulsar/broker/service/Consumer.java | 24 ++-- .../broker/service/BrokerServiceTest.java | 26 ++++ .../impl/KeySharedSubscriptionTest.java | 136 ++++++++---------- 3 files changed, 98 insertions(+), 88 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 8924b750eb624..a7c06d0c85d63 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 @@ -444,6 +444,7 @@ public CompletableFuture messageAcked(CommandAck ack) { private CompletableFuture individualAckNormal(CommandAck ack, Map properties) { List positionsAcked = new ArrayList<>(); long totalAckCount = 0; + boolean individualAck = false; for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); PositionImpl position; @@ -467,14 +468,18 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map producer = pulsarClient.newProducer() + .topic(topicName) + .create(); + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("sub-1") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .isAckReceiptEnabled(true) + .subscribe(); + producer.send("1".getBytes(StandardCharsets.UTF_8)); + Message message = consumer1.receive(); + consumer1.acknowledge(message); + consumer1.acknowledge(message); + assertEquals(admin.topics().getStats(topicName).getSubscriptions() + .get("sub-1").getUnackedMessages(), 0); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java index 213296d22833a..d2288f948b830 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java @@ -18,32 +18,35 @@ */ package org.apache.pulsar.client.impl; -import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import lombok.Cleanup; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.RandomStringUtils; +import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; -import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; @Test(groups = "broker-impl") public class KeySharedSubscriptionTest extends ProducerConsumerBase { @@ -70,91 +73,58 @@ public Object[][] subType() { @Test(dataProvider = "subType") public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType) throws PulsarClientException { - PulsarClient pulsarClient = PulsarClient.builder(). - serviceUrl(lookupUrl.toString()) - .build(); final int totalMsg = 1000; String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5); - Map, List> nameToId = Maps.newConcurrentMap(); + Map, List> nameToId = new ConcurrentHashMap<>(); Set pubMessages = Sets.newConcurrentHashSet(); Set recMessages = Sets.newConcurrentHashSet(); AtomicLong lastActiveTime = new AtomicLong(); AtomicBoolean canAcknowledgement = new AtomicBoolean(false); - @Cleanup - Consumer consumer1 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionName("sub-1") - .subscriptionType(subscriptionType) - .consumerName("con-1") - .messageListener((cons1, msg) -> { - lastActiveTime.set(System.currentTimeMillis()); - nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>()) - .add(msg.getMessageId()); - recMessages.add(msg.getMessageId()); - if (canAcknowledgement.get()) { - try { - cons1.acknowledge(msg); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } - } - }) - .subscribe(); - @Cleanup - Consumer consumer2 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionName("sub-1") - .subscriptionType(subscriptionType) - .messageListener((cons2, msg) -> { - lastActiveTime.set(System.currentTimeMillis()); - nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>()) - .add(msg.getMessageId()); - recMessages.add(msg.getMessageId()); - if (canAcknowledgement.get()) { - try { - cons2.acknowledge(msg); - } catch (PulsarClientException e) { - throw new RuntimeException(e); + List> consumerList = new ArrayList<>(); + // create 3 consumers + for (int i = 0; i < 3; i++) { + ConsumerBuilder builder = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName("sub-1") + .subscriptionType(subscriptionType) + .messageListener((consumer, msg) -> { + lastActiveTime.set(System.currentTimeMillis()); + nameToId.computeIfAbsent(consumer, (k) -> new ArrayList<>()) + .add(msg.getMessageId()); + recMessages.add(msg.getMessageId()); + if (canAcknowledgement.get()) { + try { + consumer.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } } - } - }) - .consumerName("con-2") - .subscribe(); - @Cleanup - Consumer consumer3 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionName("sub-1") - .subscriptionType(subscriptionType) - .messageListener((cons3, msg) -> { - lastActiveTime.set(System.currentTimeMillis()); - nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>()) - .add(msg.getMessageId()); - recMessages.add(msg.getMessageId()); - if (canAcknowledgement.get()) { - try { - cons3.acknowledge(msg); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } - } - }) - .consumerName("con-3") - .subscribe(); + }); + + if (subscriptionType == SubscriptionType.Key_Shared) { + // ensure every consumer can be distributed messages + int hash = Murmur3_32Hash.getInstance().makeHash(("key-" + i).getBytes()) + % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; + builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(hash, hash))); + } + + consumerList.add(builder.subscribe()); + } - @Cleanup Producer producer = pulsarClient.newProducer() .topic(topic) .enableBatching(true) .batchingMaxPublishDelay(1, TimeUnit.MILLISECONDS) // We chose 9 because the maximum unacked message is 10 .batchingMaxMessages(9) + .batcherBuilder(BatcherBuilder.KEY_BASED) .create(); for (int i = 0; i < totalMsg; i++) { - producer.sendAsync(UUID.randomUUID().toString() - .getBytes(StandardCharsets.UTF_8)) - .thenAccept(pubMessages::add); + byte[] msg = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); + producer.newMessage().key("key-" + (i % 3)).value(msg) + .sendAsync().thenAccept(pubMessages::add); } // Wait for all consumers can not read more messages. the consumers are stuck by max unacked messages. @@ -176,7 +146,7 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc // Wait for all consumers to continue receiving messages. Awaitility.await() - .atMost(30, TimeUnit.SECONDS) + .atMost(15, TimeUnit.SECONDS) .pollDelay(5, TimeUnit.SECONDS) .until(() -> (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); @@ -186,5 +156,11 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc Assert.assertEquals(pubMessages.size(), totalMsg); Assert.assertEquals(pubMessages.size(), recMessages.size()); Assert.assertTrue(recMessages.containsAll(pubMessages)); + + // cleanup + producer.close(); + for (Consumer consumer : consumerList) { + consumer.close(); + } } } From 524fae1b004ac7dd2396075cc980ac9ca1ea432b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 11 Mar 2024 15:40:54 +0200 Subject: [PATCH 11/16] [fix][ci] Tolerate mount option change failing in CI (#22241) (cherry picked from commit 46a0226efb06fba635c09ec6ed3f81cdd6b5af29) --- .github/actions/tune-runner-vm/action.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/actions/tune-runner-vm/action.yml b/.github/actions/tune-runner-vm/action.yml index e8914dbe74f6c..b30a5e0661645 100644 --- a/.github/actions/tune-runner-vm/action.yml +++ b/.github/actions/tune-runner-vm/action.yml @@ -47,8 +47,8 @@ runs: # tune filesystem mount options, https://www.kernel.org/doc/Documentation/filesystems/ext4.txt # commit=999999, effectively disables automatic syncing to disk (default is every 5 seconds) # nobarrier/barrier=0, loosen data consistency on system crash (no negative impact to empheral CI nodes) - sudo mount -o remount,nodiscard,commit=999999,barrier=0 / - sudo mount -o remount,nodiscard,commit=999999,barrier=0 /mnt + sudo mount -o remount,nodiscard,commit=999999,barrier=0 / || true + sudo mount -o remount,nodiscard,commit=999999,barrier=0 /mnt || true # disable discard/trim at device level since remount with nodiscard doesn't seem to be effective # https://www.spinics.net/lists/linux-ide/msg52562.html for i in /sys/block/sd*/queue/discard_max_bytes; do From e3bbedf32797ec1ad8294face61d9a0eeffa6cba Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 15 Mar 2024 07:41:13 -0700 Subject: [PATCH 12/16] [fix][sec] Upgrade Zookeeper to 3.9.2 to address CVE-2024-23944 (#22275) (cherry picked from commit 2ffcf62f628c6f41e78eeb2dd64999d558b6f617) --- distribution/server/src/assemble/LICENSE.bin.txt | 6 +++--- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index e10bc19bb66f9..75c25e31998e5 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -525,9 +525,9 @@ The Apache Software License, Version 2.0 - io.vertx-vertx-web-3.9.8.jar - io.vertx-vertx-web-common-3.9.8.jar * Apache ZooKeeper - - org.apache.zookeeper-zookeeper-3.9.1.jar - - org.apache.zookeeper-zookeeper-jute-3.9.1.jar - - org.apache.zookeeper-zookeeper-prometheus-metrics-3.9.1.jar + - org.apache.zookeeper-zookeeper-3.9.2.jar + - org.apache.zookeeper-zookeeper-jute-3.9.2.jar + - org.apache.zookeeper-zookeeper-prometheus-metrics-3.9.2.jar * Snappy Java - org.xerial.snappy-snappy-java-1.1.10.5.jar * Google HTTP Client diff --git a/pom.xml b/pom.xml index 65f11ba8776bf..beb3461358eed 100644 --- a/pom.xml +++ b/pom.xml @@ -105,7 +105,7 @@ flexible messaging model and an intuitive client API. 1.26.0 4.14.8 - 3.9.1 + 3.9.2 1.5.0 1.1.10.5 3.2.5 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index f8d79f6b01379..76955a58fa667 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -470,8 +470,8 @@ The Apache Software License, Version 2.0 - memory-0.8.3.jar - sketches-core-0.8.3.jar * Apache Zookeeper - - zookeeper-3.9.1.jar - - zookeeper-jute-3.9.1.jar + - zookeeper-3.9.2.jar + - zookeeper-jute-3.9.2.jar * Apache Yetus Audience Annotations - audience-annotations-0.12.0.jar * Swagger From ba895f05ee0176f4aa4e3e7f20765f98463cffb5 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 15 Mar 2024 07:56:30 -0700 Subject: [PATCH 13/16] [fix] Upgrade jose4j to 0.9.4 (#22273) (cherry picked from commit 999e39b0c7e568a9ac02fca945a558ca4adad909) # Conflicts: # distribution/server/src/assemble/LICENSE.bin.txt --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 75c25e31998e5..67914c06e9993 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -323,7 +323,7 @@ The Apache Software License, Version 2.0 * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.0.1.jar - * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.3.jar + * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson - com.google.code.gson-gson-2.8.9.jar - io.gsonfire-gson-fire-1.8.5.jar diff --git a/pom.xml b/pom.xml index beb3461358eed..07f3d4ebddbd9 100644 --- a/pom.xml +++ b/pom.xml @@ -200,6 +200,7 @@ flexible messaging model and an intuitive client API. 2.0.2 4.2.0 18.0.0 + 0.9.4 4.9.3 2.8.0 @@ -588,6 +589,12 @@ flexible messaging model and an intuitive client API. ${rocksdb.version} + + org.bitbucket.b_c + jose4j + ${jose4j.version} + + org.eclipse.jetty jetty-server From 838c0c331f6c8876c08827ac90adf3eedbf9bf8c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 20 Mar 2024 13:50:54 +0800 Subject: [PATCH 14/16] [fix] [broker] [branch-2.10] Remove the unnecessary HTTP response header: Content-Encoding when calling getSchema (#22307) --- .../admin/impl/SchemasResourceBase.java | 5 +-- .../org/apache/pulsar/schema/SchemaTest.java | 41 +++++++++++++++++++ 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java index b9533c6f29192..5bef4ccd0aa61 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.stream.Collectors; import javax.ws.rs.container.AsyncResponse; -import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; @@ -258,7 +257,7 @@ private static void handleGetSchemaResponse(AsyncResponse response, SchemaAndMet response.resume(Response.status( Response.Status.NOT_FOUND.getStatusCode(), "Schema is deleted").build()); } else { - response.resume(Response.ok().encoding(MediaType.APPLICATION_JSON) + response.resume(Response.ok() .entity(convertSchemaAndMetadataToGetSchemaResponse(schema)).build()); } } else { @@ -275,7 +274,7 @@ private static void handleGetAllSchemasResponse(AsyncResponse response, List response = new CompletableFuture(); + schemas.asyncGetRequest(path, new InvocationCallback() { + + @Override + public void completed(javax.ws.rs.core.Response getSchemaResponse) { + response.complete(getSchemaResponse); + } + + @Override + public void failed(Throwable throwable) { + response.completeExceptionally(throwable); + } + }); + MultivaluedMap responseHeaders = response.join().getHeaders(); + assertTrue(!responseHeaders.containsKey(HttpHeaders.CONTENT_ENCODING) + || !responseHeaders.get(HttpHeaders.CONTENT_ENCODING).toString().contains("application/json")); + assertTrue(responseHeaders.containsKey(HttpHeaders.CONTENT_TYPE) + && responseHeaders.get(HttpHeaders.CONTENT_TYPE).toString().contains("application/json")); + } + @EqualsAndHashCode static class User implements Serializable { private String name; From c1f4d0a8c229d79fbb9a6d30275496608aea9086 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 20 Mar 2024 13:49:54 +0800 Subject: [PATCH 15/16] [improve][broker] Add fine-grain authorization to ns/topic management endpoints (#22305) (cherry picked from commit fd34d4ab9c5aa7e0dca961d5a8badae4613fbe8e) # Conflicts: # pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java (cherry picked from commit 99eb49a68982271562597d4c4cea127132bc0b35) # Conflicts: # pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java # pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java (cherry picked from commit 5c4f4cb64167c0ea1a5775f6d991b386ad95c786) # Conflicts: # pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java # pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java # pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java --- .../pulsar/broker/admin/AdminResource.java | 7 +- .../broker/admin/impl/NamespacesBase.java | 166 ++++++----- .../admin/impl/PersistentTopicsBase.java | 137 ++++----- .../broker/admin/v2/PersistentTopics.java | 1 - .../pulsar/broker/admin/BaseAuthZTest.java | 110 +++++++ .../broker/admin/NamespaceAuthZTest.java | 117 ++++++++ .../pulsar/broker/admin/TopicAuthZTest.java | 272 ++++++++++++++++++ .../broker/admin/TopicPoliciesAuthZTest.java | 205 +++++++------ 8 files changed, 768 insertions(+), 247 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index f64e1d94507e5..6c09bd982f3c5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -54,8 +54,6 @@ import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.PolicyName; -import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscribeRate; @@ -704,10 +702,7 @@ protected static void resumeAsyncResponseExceptionally(AsyncResponse asyncRespon } protected CompletableFuture getSchemaCompatibilityStrategyAsync() { - return validateTopicPolicyOperationAsync(topicName, - PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, - PolicyOperation.READ) - .thenCompose((__) -> getSchemaCompatibilityStrategyAsyncWithoutAuth()).whenComplete((__, ex) -> { + return getSchemaCompatibilityStrategyAsyncWithoutAuth().whenComplete((__, ex) -> { if (ex != null) { log.error("[{}] Failed to get schema compatibility strategy of topic {} {}", clientAppId(), topicName, ex); 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 24d2b97fc26d8..9c859238249c9 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 @@ -2758,102 +2758,110 @@ protected void internalSetMaxTopicsPerNamespace(Integer maxTopicsPerNamespace) { } protected void internalSetProperty(String key, String value, AsyncResponse asyncResponse) { - validatePoliciesReadOnlyAccess(); - updatePoliciesAsync(namespaceName, policies -> { - policies.properties.put(key, value); - return policies; - }).thenAccept(v -> { - log.info("[{}] Successfully set property for key {} on namespace {}", clientAppId(), key, - namespaceName); - asyncResponse.resume(Response.noContent().build()); - }).exceptionally(ex -> { - Throwable cause = ex.getCause(); - log.error("[{}] Failed to set property for key {} on namespace {}", clientAppId(), key, - namespaceName, cause); - asyncResponse.resume(cause); - return null; - }); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + policies.properties.put(key, value); + return policies; + })) + .thenAccept(v -> { + log.info("[{}] Successfully set property for key {} on namespace {}", clientAppId(), key, + namespaceName); + asyncResponse.resume(Response.noContent().build()); + }).exceptionally(ex -> { + Throwable cause = ex.getCause(); + log.error("[{}] Failed to set property for key {} on namespace {}", clientAppId(), key, + namespaceName, cause); + asyncResponse.resume(cause); + return null; + }); } protected void internalSetProperties(Map properties, AsyncResponse asyncResponse) { - validatePoliciesReadOnlyAccess(); - updatePoliciesAsync(namespaceName, policies -> { - policies.properties.putAll(properties); - return policies; - }).thenAccept(v -> { - log.info("[{}] Successfully set {} properties on namespace {}", clientAppId(), properties.size(), - namespaceName); - asyncResponse.resume(Response.noContent().build()); - }).exceptionally(ex -> { - Throwable cause = ex.getCause(); - log.error("[{}] Failed to set {} properties on namespace {}", clientAppId(), properties.size(), - namespaceName, cause); - asyncResponse.resume(cause); - return null; - }); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + policies.properties.putAll(properties); + return policies; + })) + .thenAccept(v -> { + log.info("[{}] Successfully set {} properties on namespace {}", clientAppId(), properties.size(), + namespaceName); + asyncResponse.resume(Response.noContent().build()); + }).exceptionally(ex -> { + Throwable cause = ex.getCause(); + log.error("[{}] Failed to set {} properties on namespace {}", clientAppId(), properties.size(), + namespaceName, cause); + asyncResponse.resume(cause); + return null; + }); } protected void internalGetProperty(String key, AsyncResponse asyncResponse) { - getNamespacePoliciesAsync(namespaceName).thenAccept(policies -> { - asyncResponse.resume(policies.properties.get(key)); - }).exceptionally(ex -> { - Throwable cause = ex.getCause(); - log.error("[{}] Failed to get property for key {} of namespace {}", clientAppId(), key, - namespaceName, cause); - asyncResponse.resume(cause); - return null; - }); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) + .thenAccept(policies -> asyncResponse.resume(policies.properties.get(key))) + .exceptionally(ex -> { + Throwable cause = ex.getCause(); + log.error("[{}] Failed to get property for key {} of namespace {}", clientAppId(), key, + namespaceName, cause); + asyncResponse.resume(cause); + return null; + }); } protected void internalGetProperties(AsyncResponse asyncResponse) { - getNamespacePoliciesAsync(namespaceName).thenAccept(policies -> { - asyncResponse.resume(policies.properties); - }).exceptionally(ex -> { - Throwable cause = ex.getCause(); - log.error("[{}] Failed to get properties of namespace {}", clientAppId(), namespaceName, cause); - asyncResponse.resume(cause); - return null; - }); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) + .thenAccept(policies -> asyncResponse.resume(policies.properties)) + .exceptionally(ex -> { + Throwable cause = ex.getCause(); + log.error("[{}] Failed to get properties of namespace {}", clientAppId(), namespaceName, cause); + asyncResponse.resume(cause); + return null; + }); } protected void internalRemoveProperty(String key, AsyncResponse asyncResponse) { - validatePoliciesReadOnlyAccess(); - AtomicReference oldVal = new AtomicReference<>(null); - updatePoliciesAsync(namespaceName, policies -> { - oldVal.set(policies.properties.remove(key)); - return policies; - }).thenAccept(v -> { - asyncResponse.resume(oldVal.get()); - log.info("[{}] Successfully remove property for key {} on namespace {}", clientAppId(), key, - namespaceName); - }).exceptionally(ex -> { - Throwable cause = ex.getCause(); - log.error("[{}] Failed to remove property for key {} on namespace {}", clientAppId(), key, - namespaceName, cause); - asyncResponse.resume(cause); - return null; - }); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + oldVal.set(policies.properties.remove(key)); + return policies; + })).thenAccept(v -> { + asyncResponse.resume(oldVal.get()); + log.info("[{}] Successfully remove property for key {} on namespace {}", clientAppId(), key, + namespaceName); + }).exceptionally(ex -> { + Throwable cause = ex.getCause(); + log.error("[{}] Failed to remove property for key {} on namespace {}", clientAppId(), key, + namespaceName, cause); + asyncResponse.resume(cause); + return null; + }); } protected void internalClearProperties(AsyncResponse asyncResponse) { - validatePoliciesReadOnlyAccess(); AtomicReference clearedCount = new AtomicReference<>(0); - updatePoliciesAsync(namespaceName, policies -> { - clearedCount.set(policies.properties.size()); - policies.properties.clear(); - return policies; - }).thenAccept(v -> { - asyncResponse.resume(Response.noContent().build()); - log.info("[{}] Successfully clear {} properties for on namespace {}", clientAppId(), clearedCount.get(), - namespaceName); - }).exceptionally(ex -> { - Throwable cause = ex.getCause(); - log.error("[{}] Failed to remove property for key {} on namespace {}", clientAppId(), clearedCount.get(), - namespaceName, cause); - asyncResponse.resume(cause); - return null; - }); + validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + .thenCompose(__ -> updatePoliciesAsync(namespaceName, policies -> { + clearedCount.set(policies.properties.size()); + policies.properties.clear(); + return policies; + })) + .thenAccept(v -> { + asyncResponse.resume(Response.noContent().build()); + log.info("[{}] Successfully clear {} properties on namespace {}", clientAppId(), clearedCount.get(), + namespaceName); + }).exceptionally(ex -> { + Throwable cause = ex.getCause(); + log.error("[{}] Failed to clear {} properties on namespace {}", clientAppId(), clearedCount.get(), + namespaceName, cause); + asyncResponse.resume(cause); + return null; + }); } private CompletableFuture updatePoliciesAsync(NamespaceName ns, Function updateFunction) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 82711096701f2..dbcfd734a375c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -541,7 +541,9 @@ protected void internalUpdatePartitionedTopic(int numPartitions, protected void internalCreateMissedPartitions(AsyncResponse asyncResponse) { getPartitionedTopicMetadataAsync(topicName, false, false).thenAccept(metadata -> { if (metadata != null) { - tryCreatePartitionsAsync(metadata.partitions).thenAccept(v -> { + CompletableFuture future = validateNamespaceOperationAsync(topicName.getNamespaceObject(), + NamespaceOperation.CREATE_TOPIC); + future.thenCompose(__ -> tryCreatePartitionsAsync(metadata.partitions)).thenAccept(v -> { asyncResponse.resume(Response.noContent().build()); }).exceptionally(e -> { log.error("[{}] Failed to create partitions for topic {}", clientAppId(), topicName); @@ -783,13 +785,13 @@ private CompletableFuture internalRemovePartitionsAuthenticationPoliciesAs protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authoritative) { log.info("[{}] Unloading topic {}", clientAppId(), topicName); - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenAccept(__ -> { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.UNLOAD); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenAccept(__ -> { // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { if (checkTopicIsTransactionCoordinatorAssign(topicName)) { @@ -1049,13 +1051,12 @@ protected CompletableFuture internalSetDeduplicationSnapshotInterval(Integ private void internalUnloadNonPartitionedTopicAsync(AsyncResponse asyncResponse, boolean authoritative) { validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(unused -> validateTopicOperationAsync(topicName, TopicOperation.UNLOAD) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> topic.close(false)) .thenRun(() -> { log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName); asyncResponse.resume(Response.noContent().build()); - })) + }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (!isNot307And404Exception(ex)) { @@ -1068,16 +1069,14 @@ private void internalUnloadNonPartitionedTopicAsync(AsyncResponse asyncResponse, private void internalUnloadTransactionCoordinatorAsync(AsyncResponse asyncResponse, boolean authoritative) { validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.UNLOAD) - .thenCompose(v -> pulsar() - .getTransactionMetadataStoreService() - .removeTransactionMetadataStore( - TransactionCoordinatorID.get(topicName.getPartitionIndex()))) - .thenRun(() -> { - log.info("[{}] Successfully unloaded tc {}", clientAppId(), - topicName.getPartitionIndex()); - asyncResponse.resume(Response.noContent().build()); - })) + .thenCompose(v -> pulsar() + .getTransactionMetadataStoreService() + .removeTransactionMetadataStore( + TransactionCoordinatorID.get(topicName.getPartitionIndex()))) + .thenRun(() -> { + log.info("[{}] Successfully unloaded tc {}", clientAppId(), topicName.getPartitionIndex()); + asyncResponse.resume(Response.noContent().build()); + }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (!isNot307And404Exception(ex)) { @@ -1311,13 +1310,13 @@ protected PersistentTopicInternalStats internalGetInternalStats(boolean authorit } protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenAccept(__ -> { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.GET_STATS); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenAccept(__ -> { // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalGetManagedLedgerInfoForNonPartitionedTopic(asyncResponse); @@ -1423,13 +1422,13 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) { protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean authoritative, boolean perPartition, boolean getPreciseBacklog, boolean subscriptionBacklogSize, boolean getEarliestTimeInBacklog) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.GET_STATS); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)).thenAccept(partitionMetadata -> { if (partitionMetadata.partitions == 0) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned Topic not found")); @@ -1497,14 +1496,15 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean } protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenAccept(partitionMetadata -> { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.GET_STATS); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { if (partitionMetadata.partitions == 0) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned Topic not found")); return; @@ -2201,13 +2201,14 @@ private CompletableFuture internalResetCursorForNonPartitionedTopic(String protected void internalCreateSubscription(AsyncResponse asyncResponse, String subscriptionName, MessageIdImpl messageId, boolean authoritative, boolean replicated, Map properties) { - CompletableFuture ret; - if (topicName.isGlobal()) { - ret = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - ret = CompletableFuture.completedFuture(null); - } - ret.thenAccept(__ -> { + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.SUBSCRIBE, + subscriptionName); + ret.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenAccept(__ -> { final MessageIdImpl targetMessageId = messageId == null ? (MessageIdImpl) MessageId.latest : messageId; log.info("[{}][{}] Creating subscription {} at message id {} with properties {}", clientAppId(), topicName, subscriptionName, targetMessageId, properties); @@ -2366,14 +2367,13 @@ private void internalCreateSubscriptionForNonPartitionedTopic( protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse, String subName, Map subscriptionProperties, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)).thenAccept(__ -> { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.SUBSCRIBE, subName); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)).thenAccept(__ -> { if (topicName.isPartitioned()) { internalUpdateSubscriptionPropertiesForNonPartitionedTopic(asyncResponse, subName, subscriptionProperties, authoritative); @@ -3898,13 +3898,14 @@ private CompletableFuture internalExpireMessagesNonPartitionedTopicByPosit protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean authoritative) { log.info("[{}] Trigger compaction on topic {}", clientAppId(), topicName); - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenAccept(__ -> { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.COMPACT); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenAccept(__ -> { // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalTriggerCompactionNonPartitionedTopic(asyncResponse, authoritative); @@ -5135,12 +5136,12 @@ private void internalGetReplicatedSubscriptionStatusForNonPartitionedTopic(Async } protected CompletableFuture internalGetSchemaCompatibilityStrategy(boolean applied) { + CompletableFuture future = validateTopicPolicyOperationAsync(topicName, + PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ); if (applied) { - return getSchemaCompatibilityStrategyAsync(); + return future.thenCompose(__ -> getSchemaCompatibilityStrategyAsync()); } - return validateTopicPolicyOperationAsync(topicName, - PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, - PolicyOperation.READ) + return future .thenCompose(n -> getTopicPoliciesAsyncWithRetry(topicName).thenApply(op -> { if (!op.isPresent()) { return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 4dd4c1310cc62..3ceb8083e36d2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -372,7 +372,6 @@ public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @ApiParam(value = "Offload policies for the specified topic") OffloadPoliciesImpl offloadPolicies) { validateTopicName(tenant, namespace, encodedTopic); - validateTopicPolicyOperation(topicName, PolicyName.OFFLOAD, PolicyOperation.WRITE); validateTopicPolicyOperationAsync(topicName, PolicyName.OFFLOAD, PolicyOperation.WRITE) .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetOffloadPolicies(offloadPolicies, isGlobal)) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java new file mode 100644 index 0000000000000..58d8da8d16056 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java @@ -0,0 +1,110 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import io.jsonwebtoken.SignatureAlgorithm; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import javax.crypto.SecretKey; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; +import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; +import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminBuilder; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; + +public abstract class BaseAuthZTest extends MockedPulsarServiceBaseTest { + protected static final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); + private static final String TENANT_ADMIN_TOKEN = Jwts.builder() + .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); + private static final String BROKER_INTERNAL_CLIENT_SUBJECT = "broker_internal"; + private static final String BROKER_INTERNAL_CLIENT_TOKEN = Jwts.builder() + .claim("sub", BROKER_INTERNAL_CLIENT_SUBJECT).signWith(SECRET_KEY).compact(); + private static final String SUPER_USER_SUBJECT = "super-user"; + private static final String SUPER_USER_TOKEN = Jwts.builder() + .claim("sub", SUPER_USER_SUBJECT).signWith(SECRET_KEY).compact(); + private static final String NOBODY_SUBJECT = "nobody"; + private static final String NOBODY_TOKEN = Jwts.builder() + .claim("sub", NOBODY_SUBJECT).signWith(SECRET_KEY).compact(); + protected PulsarAdmin superUserAdmin; + protected PulsarAdmin tenantManagerAdmin; + + @BeforeClass + @Override + protected void setup() throws Exception { + conf.setSystemTopicEnabled(true); + conf.setTopicLevelPoliciesEnabled(true); + conf.setAuthorizationEnabled(true); + conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); + conf.setSuperUserRoles(new HashSet<>(Arrays.asList(SUPER_USER_SUBJECT, BROKER_INTERNAL_CLIENT_SUBJECT))); + conf.setAuthenticationEnabled(true); + conf.setAuthenticationProviders(new HashSet<>(Arrays.asList(AuthenticationProviderToken.class.getName()))); + // internal client + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + final Map brokerClientAuthParams = new HashMap<>(); + brokerClientAuthParams.put("token", BROKER_INTERNAL_CLIENT_TOKEN); + final String brokerClientAuthParamStr = ObjectMapperFactory.getThreadLocal() + .writeValueAsString(brokerClientAuthParams); + conf.setBrokerClientAuthenticationParameters(brokerClientAuthParamStr); + + Properties properties = conf.getProperties(); + if (properties == null) { + properties = new Properties(); + conf.setProperties(properties); + } + properties.put("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY)); + + internalSetup(); + setupDefaultTenantAndNamespace(); + + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + } + + @Override + protected void customizeNewPulsarAdminBuilder(PulsarAdminBuilder pulsarAdminBuilder) { + pulsarAdminBuilder.authentication(new AuthenticationToken(SUPER_USER_TOKEN)); + } + + @AfterClass + @Override + protected void cleanup() throws Exception { + internalCleanup(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java new file mode 100644 index 0000000000000..d64d0aa2aa183 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +public class NamespaceAuthZTest extends BaseAuthZTest { + @SneakyThrows + @Test + public void testProperties() { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test superuser + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + superUserAdmin.namespaces().setProperties(namespace, properties); + superUserAdmin.namespaces().setProperty(namespace, "key2", "value2"); + superUserAdmin.namespaces().getProperties(namespace); + superUserAdmin.namespaces().getProperty(namespace, "key2"); + superUserAdmin.namespaces().removeProperty(namespace, "key2"); + superUserAdmin.namespaces().clearProperties(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().setProperties(namespace, properties); + tenantManagerAdmin.namespaces().setProperty(namespace, "key2", "value2"); + tenantManagerAdmin.namespaces().getProperties(namespace); + tenantManagerAdmin.namespaces().getProperty(namespace, "key2"); + tenantManagerAdmin.namespaces().removeProperty(namespace, "key2"); + tenantManagerAdmin.namespaces().clearProperties(namespace); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setProperties(namespace, properties)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setProperty(namespace, "key2", "value2")); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getProperties(namespace)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getProperty(namespace, "key2")); + + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeProperty(namespace, "key2")); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearProperties(namespace)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Collections.singleton(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setProperties(namespace, properties)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setProperty(namespace, "key2", "value2")); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getProperties(namespace)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getProperty(namespace, "key2")); + + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeProperty(namespace, "key2")); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearProperties(namespace)); + + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + superUserAdmin.topics().delete(topic, true); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java new file mode 100644 index 0000000000000..cd0de3f4f6369 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -0,0 +1,272 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +public class TopicAuthZTest extends BaseAuthZTest { + + @SneakyThrows + @Test + public void testUnloadAndCompact() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createPartitionedTopic(topic, 2); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test superuser + superUserAdmin.topics().unload(topic); + superUserAdmin.topics().triggerCompaction(topic); + superUserAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false); + + // test tenant manager + tenantManagerAdmin.topics().unload(topic); + tenantManagerAdmin.topics().triggerCompaction(topic); + tenantManagerAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().unload(topic)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().triggerCompaction(topic)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false)); + + // Test only super/admin can do the operation, other auth are not permitted. + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Collections.singleton(action)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().unload(topic)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().triggerCompaction(topic)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false)); + + superUserAdmin.topics().revokePermissions(topic, subject); + } + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } + + @Test + @SneakyThrows + public void testGetManagedLedgerInfo() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createPartitionedTopic(topic, 2); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test superuser + superUserAdmin.topics().getInternalInfo(topic); + + // test tenant manager + tenantManagerAdmin.topics().getInternalInfo(topic); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getInternalInfo(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Collections.singleton(action)); + if (action == AuthAction.produce || action == AuthAction.consume) { + subAdmin.topics().getInternalInfo(topic); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getInternalInfo(topic)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } + + @Test + @SneakyThrows + public void testGetPartitionedStatsAndInternalStats() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createPartitionedTopic(topic, 2); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test superuser + superUserAdmin.topics().getPartitionedStats(topic, false); + superUserAdmin.topics().getPartitionedInternalStats(topic); + + // test tenant manager + tenantManagerAdmin.topics().getPartitionedStats(topic, false); + tenantManagerAdmin.topics().getPartitionedInternalStats(topic); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedStats(topic, false)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedInternalStats(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Collections.singleton(action)); + if (action == AuthAction.produce || action == AuthAction.consume) { + subAdmin.topics().getPartitionedStats(topic, false); + subAdmin.topics().getPartitionedInternalStats(topic); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedStats(topic, false)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedInternalStats(topic)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } + + @Test + @SneakyThrows + public void testCreateSubscriptionAndUpdateSubscriptionPropertiesAndAnalyzeSubscriptionBacklog() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createPartitionedTopic(topic, 2); + AtomicInteger suffix = new AtomicInteger(1); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().createSubscription(topic, "test-sub" + suffix.incrementAndGet(), MessageId.earliest); + + // test tenant manager + tenantManagerAdmin.topics().createSubscription(topic, "test-sub" + suffix.incrementAndGet(), MessageId.earliest); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().createSubscription(topic, "test-sub" + suffix.incrementAndGet(), MessageId.earliest)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Collections.singleton(action)); + if (action == AuthAction.consume) { + subAdmin.topics().createSubscription(topic, "test-sub" + suffix.incrementAndGet(), MessageId.earliest); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().createSubscription(topic, "test-sub" + suffix.incrementAndGet(), MessageId.earliest)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + // test UpdateSubscriptionProperties + Map properties = new HashMap<>(); + superUserAdmin.topics().createSubscription(topic, "test-sub", MessageId.earliest); + // test superuser + superUserAdmin.topics().updateSubscriptionProperties(topic, "test-sub" , properties); + + // test tenant manager + tenantManagerAdmin.topics().updateSubscriptionProperties(topic, "test-sub" , properties); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Collections.singleton(action)); + if (action == AuthAction.consume) { + subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } + + @Test + @SneakyThrows + public void testCreateMissingPartition() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createPartitionedTopic(topic, 2); + AtomicInteger suffix = new AtomicInteger(1); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(pulsar.getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().createMissedPartitions(topic); + + // test tenant manager + tenantManagerAdmin.topics().createMissedPartitions(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().createMissedPartitions(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Collections.singleton(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().createMissedPartitions(topic)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java index b76c6e3f3f2ec..152c84b854be5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java @@ -19,107 +19,23 @@ package org.apache.pulsar.broker.admin; import static org.awaitility.Awaitility.await; -import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; -import io.jsonwebtoken.SignatureAlgorithm; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; +import java.util.Collections; import java.util.UUID; -import javax.crypto.SecretKey; import lombok.Cleanup; import lombok.SneakyThrows; -import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; -import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; -import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; -import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -public final class TopicPoliciesAuthZTest extends MockedPulsarServiceBaseTest { - - private PulsarAdmin superUserAdmin; - - private PulsarAdmin tenantManagerAdmin; - - private static final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); - private static final String TENANT_ADMIN_TOKEN = Jwts.builder() - .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); - - - private static final String BROKER_INTERNAL_CLIENT_SUBJECT = "broker_internal"; - private static final String BROKER_INTERNAL_CLIENT_TOKEN = Jwts.builder() - .claim("sub", BROKER_INTERNAL_CLIENT_SUBJECT).signWith(SECRET_KEY).compact(); - private static final String SUPER_USER_SUBJECT = "super-user"; - private static final String SUPER_USER_TOKEN = Jwts.builder() - .claim("sub", SUPER_USER_SUBJECT).signWith(SECRET_KEY).compact(); - private static final String NOBODY_SUBJECT = "nobody"; - private static final String NOBODY_TOKEN = Jwts.builder() - .claim("sub", NOBODY_SUBJECT).signWith(SECRET_KEY).compact(); - - - @BeforeClass - @Override - protected void setup() throws Exception { - conf.setAuthorizationEnabled(true); - conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); - conf.setSuperUserRoles(Sets.newHashSet(SUPER_USER_SUBJECT, BROKER_INTERNAL_CLIENT_SUBJECT)); - conf.setAuthenticationEnabled(true); - conf.setSystemTopicEnabled(true); - conf.setTopicLevelPoliciesEnabled(true); - conf.setAuthenticationProviders(Sets.newHashSet(AuthenticationProviderToken.class.getName())); - // internal client - conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); - final Map brokerClientAuthParams = new HashMap<>(); - brokerClientAuthParams.put("token", BROKER_INTERNAL_CLIENT_TOKEN); - final String brokerClientAuthParamStr = ObjectMapperFactory.getThreadLocal() - .writeValueAsString(brokerClientAuthParams); - conf.setBrokerClientAuthenticationParameters(brokerClientAuthParamStr); - - Properties properties = conf.getProperties(); - if (properties == null) { - properties = new Properties(); - conf.setProperties(properties); - } - properties.put("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY)); - - internalSetup(); - setupDefaultTenantAndNamespace(); - - this.superUserAdmin =PulsarAdmin.builder() - .serviceHttpUrl(pulsar.getWebServiceAddress()) - .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) - .build(); - final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); - tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); - superUserAdmin.tenants().updateTenant("public", tenantInfo); - this.tenantManagerAdmin = PulsarAdmin.builder() - .serviceHttpUrl(pulsar.getWebServiceAddress()) - .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) - .build(); - } - - @Override - protected void customizeNewPulsarAdminBuilder(PulsarAdminBuilder pulsarAdminBuilder) { - pulsarAdminBuilder.authentication(new AuthenticationToken(SUPER_USER_TOKEN)); - } - @AfterClass - @Override - protected void cleanup() throws Exception { - internalCleanup(); - } +public final class TopicPoliciesAuthZTest extends BaseAuthZTest { @SneakyThrows @@ -193,7 +109,7 @@ public void testRetention() { // test sub user with permissions for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", - subject, Sets.newHashSet(action)); + subject, Collections.singleton(action)); try { subAdmin.topicPolicies().getRetention(topic); Assert.fail("unexpected behaviour"); @@ -218,6 +134,109 @@ public void testRetention() { superUserAdmin.namespaces().revokePermissionsOnNamespace("public/default", subject); } } + @SneakyThrows + @Test + public void testOffloadPolicy() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsar().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // mocked data + final OffloadPoliciesImpl definedOffloadPolicies = new OffloadPoliciesImpl(); + definedOffloadPolicies.setManagedLedgerOffloadThresholdInBytes(100L); + definedOffloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(200L); + definedOffloadPolicies.setManagedLedgerOffloadDriver(""); // set to blank value to test the behaviour + definedOffloadPolicies.setManagedLedgerOffloadBucket("buck"); + + // test superuser + superUserAdmin.topicPolicies().setOffloadPolicies(topic, definedOffloadPolicies); + + // because the topic policies is eventual consistency, we should wait here + await().untilAsserted(() -> { + final OffloadPolicies offloadPolicy = superUserAdmin.topicPolicies().getOffloadPolicies(topic); + Assert.assertEquals(offloadPolicy, definedOffloadPolicies); + }); + superUserAdmin.topicPolicies().removeOffloadPolicies(topic); + + await().untilAsserted(() -> { + final OffloadPolicies offloadPolicy = superUserAdmin.topicPolicies().getOffloadPolicies(topic); + Assert.assertNull(offloadPolicy); + }); + + // test tenant manager + + tenantManagerAdmin.topicPolicies().setOffloadPolicies(topic, definedOffloadPolicies); + await().untilAsserted(() -> { + final OffloadPolicies offloadPolicy = tenantManagerAdmin.topicPolicies().getOffloadPolicies(topic); + Assert.assertEquals(offloadPolicy, definedOffloadPolicies); + }); + tenantManagerAdmin.topicPolicies().removeOffloadPolicies(topic); + await().untilAsserted(() -> { + final OffloadPolicies offloadPolicy = tenantManagerAdmin.topicPolicies().getOffloadPolicies(topic); + Assert.assertNull(offloadPolicy); + }); + + // test nobody + + try { + subAdmin.topicPolicies().getOffloadPolicies(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setOffloadPolicies(topic, definedOffloadPolicies); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeOffloadPolicies(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + // test sub user with permissions + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", + subject, Collections.singleton(action)); + try { + subAdmin.topicPolicies().getOffloadPolicies(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setOffloadPolicies(topic, definedOffloadPolicies); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeOffloadPolicies(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace("public/default", subject); + } + } @SneakyThrows @Test @@ -230,7 +249,7 @@ public void testMaxUnackedMessagesOnConsumer() { superUserAdmin.topics().createNonPartitionedTopic(topic); @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() - .serviceHttpUrl(pulsar.getWebServiceAddress()) + .serviceHttpUrl(getPulsar().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -293,7 +312,7 @@ public void testMaxUnackedMessagesOnConsumer() { // test sub user with permissions for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", - subject, Sets.newHashSet(action)); + subject, Collections.singleton(action)); try { subAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic); Assert.fail("unexpected behaviour"); @@ -330,7 +349,7 @@ public void testMaxUnackedMessagesOnSubscription() { superUserAdmin.topics().createNonPartitionedTopic(topic); @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() - .serviceHttpUrl(pulsar.getWebServiceAddress()) + .serviceHttpUrl(getPulsar().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -395,7 +414,7 @@ public void testMaxUnackedMessagesOnSubscription() { // test sub user with permissions for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", - subject, Sets.newHashSet(action)); + subject, Collections.singleton(action)); try { subAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic); Assert.fail("unexpected behaviour"); From 83d996b6b2bfa9f730bf7352f0e8f7fda5fde373 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 20 Mar 2024 17:47:09 +0200 Subject: [PATCH 16/16] Add cleanup to BaseAuthzTest used in backport of PR #22305 (cherry picked from commit 91ee5807d1bf6d2b9ea393c768c9fe5eae1f71e2) --- .../apache/pulsar/broker/admin/BaseAuthZTest.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java index 58d8da8d16056..f5d82b42fc901 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BaseAuthZTest.java @@ -56,7 +56,7 @@ public abstract class BaseAuthZTest extends MockedPulsarServiceBaseTest { protected PulsarAdmin superUserAdmin; protected PulsarAdmin tenantManagerAdmin; - @BeforeClass + @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { conf.setSystemTopicEnabled(true); @@ -102,9 +102,17 @@ protected void customizeNewPulsarAdminBuilder(PulsarAdminBuilder pulsarAdminBuil pulsarAdminBuilder.authentication(new AuthenticationToken(SUPER_USER_TOKEN)); } - @AfterClass + @AfterClass(alwaysRun = true) @Override protected void cleanup() throws Exception { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + if (tenantManagerAdmin != null) { + tenantManagerAdmin.close(); + tenantManagerAdmin = null; + } internalCleanup(); } }