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 cffe295eab442..497eec5ba9d44 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; @@ -703,10 +701,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 a123efacbd04a..2aa921d2e13fa 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 @@ -2568,102 +2568,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 91758668be514..ef544365c65d1 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 @@ -547,7 +547,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); @@ -797,13 +799,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)) { @@ -1063,13 +1065,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 (!isRedirectException(ex)) { @@ -1082,16 +1083,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 (!isRedirectException(ex)) { @@ -1318,13 +1317,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); @@ -1430,13 +1429,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")); @@ -1504,14 +1503,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; @@ -2291,13 +2291,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); @@ -2457,14 +2458,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); @@ -2535,14 +2535,13 @@ protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse, protected void internalAnalyzeSubscriptionBacklog(AsyncResponse asyncResponse, String subName, Optional position, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.CONSUME, subName); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> { if (topicName.isPartitioned()) { return CompletableFuture.completedFuture(null); @@ -2574,14 +2573,13 @@ protected void internalAnalyzeSubscriptionBacklog(AsyncResponse asyncResponse, S protected void internalGetSubscriptionProperties(AsyncResponse asyncResponse, String subName, 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.CONSUME, subName); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } + return CompletableFuture.completedFuture(null); + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)).thenAccept(__ -> { if (topicName.isPartitioned()) { internalGetSubscriptionPropertiesForNonPartitionedTopic(asyncResponse, subName, authoritative); @@ -4097,13 +4095,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); @@ -5390,12 +5389,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/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..92d69ec717107 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -0,0 +1,252 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.broker.admin; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; +import io.jsonwebtoken.Jwts; +import io.jsonwebtoken.SignatureAlgorithm; +import lombok.Cleanup; +import lombok.Getter; +import lombok.SneakyThrows; +import org.apache.pulsar.broker.PulsarService; +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.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.testng.Assert; +import org.testng.annotations.*; + +import javax.crypto.SecretKey; +import java.util.*; +import java.util.Optional; + +@Test(groups = "broker-admin") +public class NamespaceAuthZTest extends MockedPulsarServiceBaseTest{ + + private PulsarAdmin superUserAdmin; + private PulsarAdmin tenantManagerAdmin; + @Getter + private PulsarService pulsarService; + private PulsarAdmin serviceInternalAdmin; + + 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(); + + protected static final String SUPER_USER_SUBJECT = "super-user"; + + protected static final String SUPER_USER_TOKEN = Jwts.builder() + .claim("sub", SUPER_USER_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(); + + protected static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final String DEFAULT_TENANT = "public"; + private static final String DEFAULT_NAMESPACE = "public/default"; + private static final String TEST_CLUSTER_NAME = "test-standalone"; + + { + conf.setClusterName(TEST_CLUSTER_NAME); + conf.setBrokerShutdownTimeoutMs(0L); + conf.setBrokerServicePort(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setAdvertisedAddress("localhost"); + conf.setWebServicePort(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); + conf.setNumExecutorThreadPoolSize(5); + conf.setExposeBundlesMetricsInPrometheus(true); + } + + @Override + @BeforeClass + protected void setup() throws Exception { + // enable auth&auth and use JWT at broker + conf.setAuthenticationEnabled(true); + conf.setAuthorizationEnabled(true); + conf.getProperties().setProperty("tokenSecretKey", "data:;base64," + + Base64.getEncoder().encodeToString(SECRET_KEY.getEncoded())); + Set superUserRoles = new HashSet<>(); + superUserRoles.add("admin"); + conf.setSuperUserRoles(superUserRoles); + Set providers = new HashSet<>(); + providers.add(AuthenticationProviderToken.class.getName()); + conf.setAuthenticationProviders(providers); + configureTokenAuthentication(); + configureDefaultAuthorization(); + super.internalSetup(); + this.pulsarService = getPulsar(); + this.serviceInternalAdmin = pulsarService.getAdminClient(); + setupDefaultTenantAndNamespace(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + } + + protected void setupDefaultTenantAndNamespace() throws Exception { + if (!serviceInternalAdmin.clusters().getClusters().contains(TEST_CLUSTER_NAME)) { + serviceInternalAdmin.clusters().createCluster(TEST_CLUSTER_NAME, + ClusterData.builder().serviceUrl(pulsarService.getWebServiceAddress()).build()); + } + if (!serviceInternalAdmin.tenants().getTenants().contains(DEFAULT_TENANT)) { + serviceInternalAdmin.tenants().createTenant(DEFAULT_TENANT, TenantInfo.builder().allowedClusters( + Sets.newHashSet(TEST_CLUSTER_NAME)).build()); + } + if (!serviceInternalAdmin.namespaces().getNamespaces(DEFAULT_TENANT).contains(DEFAULT_NAMESPACE)) { + serviceInternalAdmin.namespaces().createNamespace(DEFAULT_NAMESPACE); + } + } + + @SneakyThrows + protected void configureTokenAuthentication() { + conf.setAuthenticationEnabled(true); + Set authenticationProviders = new HashSet<>(); + authenticationProviders.add(AuthenticationProviderToken.class.getName()); + conf.setAuthenticationProviders(authenticationProviders); + // internal client + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + final Map brokerClientAuthParams = new HashMap<>(); + brokerClientAuthParams.put("token", BROKER_INTERNAL_CLIENT_TOKEN); + final String brokerClientAuthParamStr = MAPPER.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)); + + } + + protected void configureDefaultAuthorization() { + conf.setAuthorizationEnabled(true); + conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); + Set superUserRoles = new HashSet<>(); + superUserRoles.add(SUPER_USER_SUBJECT); + superUserRoles.add(BROKER_INTERNAL_CLIENT_SUBJECT); + conf.setSuperUserRoles(superUserRoles); + } + + @Override + @AfterClass + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @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(getPulsarService().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()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, actions); + 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..7df997cf45feb --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -0,0 +1,446 @@ +/** + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; +import io.jsonwebtoken.Jwts; +import io.jsonwebtoken.SignatureAlgorithm; +import lombok.Cleanup; +import lombok.Getter; +import lombok.SneakyThrows; +import org.apache.pulsar.broker.PulsarService; +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.PulsarAdminException; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.testng.Assert; +import org.testng.annotations.*; + +import javax.crypto.SecretKey; +import java.util.*; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; + +@Test(groups = "broker-admin") +public class TopicAuthZTest extends MockedPulsarServiceBaseTest { + + private PulsarAdmin superUserAdmin; + private PulsarAdmin tenantManagerAdmin; + @Getter + private PulsarService pulsarService; + private PulsarAdmin serviceInternalAdmin; + + 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(); + + protected static final String SUPER_USER_SUBJECT = "super-user"; + + protected static final String SUPER_USER_TOKEN = Jwts.builder() + .claim("sub", SUPER_USER_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(); + + protected static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final String DEFAULT_TENANT = "public"; + private static final String DEFAULT_NAMESPACE = "public/default"; + private static final String TEST_CLUSTER_NAME = "test-standalone"; + + { + conf.setClusterName(TEST_CLUSTER_NAME); + conf.setBrokerShutdownTimeoutMs(0L); + conf.setBrokerServicePort(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setAdvertisedAddress("localhost"); + conf.setWebServicePort(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); + conf.setNumExecutorThreadPoolSize(5); + conf.setExposeBundlesMetricsInPrometheus(true); + } + + @Override + @BeforeClass + protected void setup() throws Exception { + // enable auth&auth and use JWT at broker + conf.setAuthenticationEnabled(true); + conf.setAuthorizationEnabled(true); + conf.getProperties().setProperty("tokenSecretKey", "data:;base64," + + Base64.getEncoder().encodeToString(SECRET_KEY.getEncoded())); + Set superUserRoles = new HashSet<>(); + superUserRoles.add("admin"); + conf.setSuperUserRoles(superUserRoles); + Set providers = new HashSet<>(); + providers.add(AuthenticationProviderToken.class.getName()); + conf.setAuthenticationProviders(providers); + conf.setTopicLevelPoliciesEnabled(true); + conf.setSystemTopicEnabled(true); + configureTokenAuthentication(); + configureDefaultAuthorization(); + super.internalSetup(); + this.pulsarService = getPulsar(); + this.serviceInternalAdmin = pulsarService.getAdminClient(); + setupDefaultTenantAndNamespace(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + } + + protected void setupDefaultTenantAndNamespace() throws Exception { + if (!serviceInternalAdmin.clusters().getClusters().contains(TEST_CLUSTER_NAME)) { + serviceInternalAdmin.clusters().createCluster(TEST_CLUSTER_NAME, + ClusterData.builder().serviceUrl(pulsarService.getWebServiceAddress()).build()); + } + if (!serviceInternalAdmin.tenants().getTenants().contains(DEFAULT_TENANT)) { + serviceInternalAdmin.tenants().createTenant(DEFAULT_TENANT, TenantInfo.builder().allowedClusters( + Sets.newHashSet(TEST_CLUSTER_NAME)).build()); + } + if (!serviceInternalAdmin.namespaces().getNamespaces(DEFAULT_TENANT).contains(DEFAULT_NAMESPACE)) { + serviceInternalAdmin.namespaces().createNamespace(DEFAULT_NAMESPACE); + } + } + + @SneakyThrows + protected void configureTokenAuthentication() { + conf.setAuthenticationEnabled(true); + Set authenticationProviders = new HashSet<>(); + authenticationProviders.add(AuthenticationProviderToken.class.getName()); + conf.setAuthenticationProviders(authenticationProviders); + // internal client + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + final Map brokerClientAuthParams = new HashMap<>(); + brokerClientAuthParams.put("token", BROKER_INTERNAL_CLIENT_TOKEN); + final String brokerClientAuthParamStr = MAPPER.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)); + + } + + protected void configureDefaultAuthorization() { + conf.setAuthorizationEnabled(true); + conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); + Set superUserRoles = new HashSet<>(); + superUserRoles.add(SUPER_USER_SUBJECT); + superUserRoles.add(BROKER_INTERNAL_CLIENT_SUBJECT); + conf.setSuperUserRoles(superUserRoles); + } + + @Override + @AfterClass + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + + @SneakyThrows + @Test + public void testUnloadAndCompactAndTrim() { + 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(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test superuser + superUserAdmin.topics().unload(topic); + superUserAdmin.topics().triggerCompaction(topic); +// superUserAdmin.topics().trimTopic(TopicName.get(topic).getPartition(0).getLocalName()); + superUserAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false); + + // test tenant manager + tenantManagerAdmin.topics().unload(topic); + tenantManagerAdmin.topics().triggerCompaction(topic); +// tenantManagerAdmin.topics().trimTopic(TopicName.get(topic).getPartition(0).getLocalName()); + 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.topics().trimTopic(TopicName.get(topic).getPartition(0).getLocalName())); + + 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()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.topics().grantPermission(topic, subject, actions); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().unload(topic)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().triggerCompaction(topic)); + +// Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, +// () -> subAdmin.topics().trimTopic(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(getPulsarService().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()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.topics().grantPermission(topic, subject, actions); + 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(getPulsarService().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()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.topics().grantPermission(topic, subject, actions); + 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(getPulsarService().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()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.topics().grantPermission(topic, subject, actions); + 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); + superUserAdmin.topics().getSubscriptionProperties(topic, "test-sub"); + superUserAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty()); + + // test tenant manager + tenantManagerAdmin.topics().updateSubscriptionProperties(topic, "test-sub" , properties); + tenantManagerAdmin.topics().getSubscriptionProperties(topic, "test-sub"); + tenantManagerAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty()); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getSubscriptionProperties(topic, "test-sub")); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty())); + + for (AuthAction action : AuthAction.values()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.topics().grantPermission(topic, subject, actions); + if (action == AuthAction.consume) { + subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties); + subAdmin.topics().getSubscriptionProperties(topic, "test-sub"); + subAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty()); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getSubscriptionProperties(topic, "test-sub")); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty())); + } + 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(getPulsarService().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()) { + Set actions = new HashSet<>(); + actions.add(action); + superUserAdmin.topics().grantPermission(topic, subject, actions); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().createMissedPartitions(topic)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java index c288c054bb9f2..0cfdc77b031d0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java @@ -124,7 +124,8 @@ private static Provider loadConscryptProvider() { conscryptClazz = Class.forName("org.conscrypt.Conscrypt"); conscryptClazz.getMethod("checkAvailability").invoke(null); } catch (Throwable e) { - log.warn("Conscrypt isn't available. Using JDK default security provider.", e); + log.warn("Conscrypt isn't available. Using JDK default security provider." + + " Cause : {}, Reason : {}", e.getCause(), e.getMessage()); return null; }