This is an automated email from the ASF dual-hosted git repository.
bogong pushed a commit to branch branch-2.9
in repository https://gitbox.apache.org/repos/asf/pulsar.git
The following commit(s) were added to refs/heads/branch-2.9 by this push:
new 863ea603abb fix can not revoke permission after update topic partition
(#17608)
863ea603abb is described below
commit 863ea603abba638ddce07d859bda35cbd8ed3b05
Author: ken <[email protected]>
AuthorDate: Mon Nov 14 10:57:01 2022 +0800
fix can not revoke permission after update topic partition (#17608)
### Motivation
cherry-pick #17393 to branch-2.9
---
.../broker/admin/impl/PersistentTopicsBase.java | 12 ++++++++----
.../server/ProxyWithJwtAuthorizationTest.java | 22 ++++++++++++++++++++++
2 files changed, 30 insertions(+), 4 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 0ee757bf9e9..5e89a8aeffe 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
@@ -310,7 +310,7 @@ public class PersistentTopicsBase extends AdminResource {
}
}
- private void revokePermissions(String topicUri, String role) {
+ private void revokePermissions(String topicUri, String role, boolean
force) {
Policies policies;
try {
policies = namespaceResources().getPolicies(namespaceName)
@@ -323,7 +323,11 @@ public class PersistentTopicsBase extends AdminResource {
||
!policies.auth_policies.getTopicAuthentication().get(topicUri).containsKey(role))
{
log.warn("[{}] Failed to revoke permission from role {} on topic:
Not set at topic level {}", clientAppId(),
role, topicUri);
- throw new RestException(Status.PRECONDITION_FAILED, "Permissions
are not set at the topic level");
+ if (force) {
+ return;
+ } else {
+ throw new RestException(Status.PRECONDITION_FAILED,
"Permissions are not set at the topic level");
+ }
}
try {
// Write the new policies to metadata store
@@ -355,10 +359,10 @@ public class PersistentTopicsBase extends AdminResource {
if (numPartitions > 0) {
for (int i = 0; i < numPartitions; i++) {
TopicName topicNamePartition = topicName.getPartition(i);
- revokePermissions(topicNamePartition.toString(), role);
+ revokePermissions(topicNamePartition.toString(), role, true);
}
}
- revokePermissions(topicName.toString(), role);
+ revokePermissions(topicName.toString(), role, false);
}
protected void internalCreateNonPartitionedTopic(boolean authoritative) {
diff --git
a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
index 6178454dd19..4a4a35ece69 100644
---
a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
+++
b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java
@@ -210,6 +210,8 @@ public class ProxyWithJwtAuthorizationTest extends
ProducerConsumerBase {
* 2. Update the topic partition number to 4.
* 3. Use new producer/consumer with client role to process the topic.
* 4. Broker should authorize producer/consumer normally.
+ * 5. revoke produce/consumer permission of topic
+ * 6. new producer/consumer should not be authorized
* </pre>
*/
@Test
@@ -291,6 +293,26 @@ public class ProxyWithJwtAuthorizationTest extends
ProducerConsumerBase {
Assert.assertEquals(messageSet, receivedMessageSet);
consumer.close();
producer.close();
+
+ // revoke produce/consume permission
+ admin.topics().revokePermissions(topicName, CLIENT_ROLE);
+
+ // produce/consume the topic should fail
+ try {
+ consumer = proxyClient.newConsumer()
+ .topic(topicName)
+ .subscriptionName(subscriptionName).subscribe();
+ Assert.fail("Should not pass");
+ } catch (PulsarClientException.AuthorizationException ex) {
+ // ok
+ }
+ try {
+ producer = proxyClient.newProducer(Schema.BYTES)
+ .topic(topicName).create();
+ Assert.fail("Should not pass");
+ } catch (PulsarClientException.AuthorizationException ex) {
+ // ok
+ }
log.info("-- Exiting {} test --", methodName);
}