You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/07/26 07:08:32 UTC

[GitHub] [pulsar] TakaHiR07 opened a new pull request, #16792: Enhance grant permission on topic

TakaHiR07 opened a new pull request, #16792:
URL: https://github.com/apache/pulsar/pull/16792

   ### Motivation
   related to https://github.com/apache/pulsar/issues/16768
   
   
   ### Modifications
   1. no need to grant permission for each partition in internalGrantPermissionsOnTopic(), only grant permission on partitionedTopicName
   2. do not revoke permission for each partition in internalRevokePermissionsOnTopic(), only revoke permission on partitionedTopicName
   3. internalGetPermissionsOnTopic() should get the partitionedTopic permission
   
   If create topic and grant permission before and revoke permission after this pr, the permission of topic partition would remain in zk metadata. This do not effect the authorization. When do delete topic operation, the remained metadata can be cleaned. 
   
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API: no
     - The schema: no
     - The default values of configurations: no
     - The wire protocol: no
     - The rest endpoints: no
     - The admin cli options: no
     - Anything that affects deployment: no
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [x] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937450225


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -319,21 +319,12 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
                                                    Set<AuthAction> actions) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
+                    CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+                    return future.thenCompose(unused ->

Review Comment:
   ```suggestion
                       return grantPermissionsAsync(topicName, role, actions)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r962443913


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -271,6 +271,24 @@ protected CompletableFuture<Map<String, Set<AuthAction>>> internalGetPermissions
                         }
                     }
                 }
+
+                // If topic is partitioned, add based topic permission
+                if (topicName.isPartitioned() && auth.getTopicAuthentication().containsKey(
+                        topicName.getPartitionedTopicName())) {
+                    for (Map.Entry<String, Set<AuthAction>> entry :
+                            auth.getTopicAuthentication().get(topicName.getPartitionedTopicName()).entrySet()) {
+                        String role = entry.getKey();
+                        Set<AuthAction> topicPermissions = entry.getValue();
+
+                        if (!permissions.containsKey(role)) {
+                            permissions.put(role, topicPermissions);
+                        } else {
+                            // Do the union between namespace and topic level
+                            Set<AuthAction> union = Sets.union(permissions.get(role), topicPermissions);
+                            permissions.put(role, union);
+                        }
+                    }
+                }

Review Comment:
   I think the `AuthorizationProvider` should provide the get permissions method to avoid this case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937461637


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -368,22 +359,12 @@ private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String r
     protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-            getPartitionedTopicMetadataAsync(topicName, true, false)
-                .thenCompose(metadata -> {
-                    int numPartitions = metadata.partitions;
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
                     CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                    if (numPartitions > 0) {
-                        for (int i = 0; i < numPartitions; i++) {
-                            TopicName topicNamePartition = topicName.getPartition(i);
-                            future = future.thenComposeAsync(unused ->
-                                    revokePermissionsAsync(topicNamePartition.toString(), role));
-                        }
-                    }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return future.thenComposeAsync(unused ->

Review Comment:
   ```suggestion
                       return revokePermissionsAsync(topicName.getPartitionedTopicName(), role)
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -368,22 +359,12 @@ private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String r
     protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-            getPartitionedTopicMetadataAsync(topicName, true, false)
-                .thenCompose(metadata -> {
-                    int numPartitions = metadata.partitions;
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
                     CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                    if (numPartitions > 0) {
-                        for (int i = 0; i < numPartitions; i++) {
-                            TopicName topicNamePartition = topicName.getPartition(i);
-                            future = future.thenComposeAsync(unused ->
-                                    revokePermissionsAsync(topicNamePartition.toString(), role));
-                        }
-                    }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return future.thenComposeAsync(unused ->

Review Comment:
   ```suggestion
                       return revokePermissionsAsync(topicName, role)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1204960025

   Hi @TakaHiR07, Thanks for your contribution! I think you should consider the compatibility here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r962454929


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -271,6 +271,24 @@ protected CompletableFuture<Map<String, Set<AuthAction>>> internalGetPermissions
                         }
                     }
                 }
+
+                // If topic is partitioned, add based topic permission
+                if (topicName.isPartitioned() && auth.getTopicAuthentication().containsKey(
+                        topicName.getPartitionedTopicName())) {
+                    for (Map.Entry<String, Set<AuthAction>> entry :
+                            auth.getTopicAuthentication().get(topicName.getPartitionedTopicName()).entrySet()) {
+                        String role = entry.getKey();
+                        Set<AuthAction> topicPermissions = entry.getValue();
+
+                        if (!permissions.containsKey(role)) {
+                            permissions.put(role, topicPermissions);
+                        } else {
+                            // Do the union between namespace and topic level
+                            Set<AuthAction> union = Sets.union(permissions.get(role), topicPermissions);
+                            permissions.put(role, union);
+                        }
+                    }
+                }

Review Comment:
   The implementation of grant is in AuthorizationProvider, while get/revoke implementation is in  persistentTopicsBase. 
   Is it better to provide the get/revoke permission method in AuthorizationProvider?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r938399106


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   > I mean that both can produce messages.
   
   Actually in pulsar old version, this situation can only produce to one partition.
   
   If we just authorized "pulsar-admin topics permissions persistent://my-tenant/my-namespace/my-topic-partition-0" in old version, only the 0 partition has permission, and the base topic has no permission, therefore msg can only be produced to partition 0, but cannot be produced to base topic.
   
   So if we do compatible with the behavior of the old version, it should only can produce to partition 0, but not have permission to produce to base topic? Or do you think that both should be able to produced in the new version?
   
   Besides, If we authorized "pulsar-admin topics permissions persistent://my-tenant/my-namespace/my-topic" in old version, both the base topic and the two partitions will have permissions. Then this pr use topicName.getPartitionedTopicName() to grant, revoke and check. No matter whether the request is the partition "my-topic-partition-0" or the base topic "my-topic", it will be mapped to base topic "my-topic". Since old version has base topic permission, I think this can satisfy the compatibility with old version.
   
   Thank you for your review and Please take a look about it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 closed pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 closed pull request #16792: fix can not revoke permission after update topic partition
URL: https://github.com/apache/pulsar/pull/16792


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r962443331


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                        grantPermissionsAsync(TopicName.get(topicName.getPartitionedTopicName()), role, actions)

Review Comment:
   Agreed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937449050


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -368,22 +359,12 @@ private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String r
     protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-            getPartitionedTopicMetadataAsync(topicName, true, false)
-                .thenCompose(metadata -> {
-                    int numPartitions = metadata.partitions;
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
                     CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                    if (numPartitions > 0) {
-                        for (int i = 0; i < numPartitions; i++) {
-                            TopicName topicNamePartition = topicName.getPartition(i);
-                            future = future.thenComposeAsync(unused ->
-                                    revokePermissionsAsync(topicNamePartition.toString(), role));
-                        }
-                    }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return future.thenComposeAsync(unused ->

Review Comment:
   ```suggestion
                       return revokePermissionsAsync(topicName, role)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1203636440

   @nodece @Technoboy- @codelipenghui PTAL, this problem would result in permission can not be revoked


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1219114676

   I have changed the proposal and I think it is the better way to solve this problem and meet the compatibility.
   
   
   pulsar old version
   |  | partition (e.g., persistent://public/default/mytopic-partition-0) | based topic (e.g., persistent://public/default/mytopic)|
   | ------ | ------ | ------ |
   | grant | only partition | grant all partition and then grant based topic |
   | revoke | only partition | revoke all partition and then revoke based topic (shutdown if throw exception) |
   | get | only partition | only based topic |
   | check | check partition and then check based topic | only based topic |
   
   pulsar new version
   |  | partition (e.g., persistent://public/default/mytopic-partition-0) | based topic (e.g., persistent://public/default/mytopic)|
   | ------ | ------ | ------ |
   | grant | only partition | only based topic |
   | revoke | only partition | revoke based topic and then revoke partition  |
   | get | only partition | only based topic |
   | check | check partition and then check based topic | only based topic |
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937449050


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -368,22 +359,12 @@ private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String r
     protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-            getPartitionedTopicMetadataAsync(topicName, true, false)
-                .thenCompose(metadata -> {
-                    int numPartitions = metadata.partitions;
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
                     CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                    if (numPartitions > 0) {
-                        for (int i = 0; i < numPartitions; i++) {
-                            TopicName topicNamePartition = topicName.getPartition(i);
-                            future = future.thenComposeAsync(unused ->
-                                    revokePermissionsAsync(topicNamePartition.toString(), role));
-                        }
-                    }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return future.thenComposeAsync(unused ->

Review Comment:
   ```suggestion
                       return revokePermissionsAsync(topicName.getPartitionedTopicName(), role)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1232753037

   It looks like you didn't update the `org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider#checkPermission`.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1234423096

   Guide: https://pulsar.apache.org/contributing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1232759979

   > It looks like you didn't update the `org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider#checkPermission`.
   
   Yes,  I didn't update this for compatibility. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] michaeljmarshall commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r960227112


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }

Review Comment:
   In order to make this change backwards compatible, I think we need to leave this block (at least in the PR that is focused on fixing the historical bug). Otherwise, custom implementations of the `AuthorizationProvider` interface would have the behavior broken on them, which shouldn't happen in a patch release.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -271,6 +271,24 @@ protected CompletableFuture<Map<String, Set<AuthAction>>> internalGetPermissions
                         }
                     }
                 }
+
+                // If topic is partitioned, add based topic permission
+                if (topicName.isPartitioned() && auth.getTopicAuthentication().containsKey(
+                        topicName.getPartitionedTopicName())) {
+                    for (Map.Entry<String, Set<AuthAction>> entry :
+                            auth.getTopicAuthentication().get(topicName.getPartitionedTopicName()).entrySet()) {
+                        String role = entry.getKey();
+                        Set<AuthAction> topicPermissions = entry.getValue();
+
+                        if (!permissions.containsKey(role)) {
+                            permissions.put(role, topicPermissions);
+                        } else {
+                            // Do the union between namespace and topic level
+                            Set<AuthAction> union = Sets.union(permissions.get(role), topicPermissions);
+                            permissions.put(role, union);
+                        }
+                    }
+                }

Review Comment:
   I am guessing this is meant to grant permission based on a topic's partition name. This code block will be insufficient. We'll need to look at the `AuthorizationProvider` to get a complete implementation.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                        grantPermissionsAsync(TopicName.get(topicName.getPartitionedTopicName()), role, actions)

Review Comment:
   Instead of passing the topic name without any of the partition information. I think we should instead grant permission to the result `topicName`, and then we can leave it up to the `AuthorizationProvider` to determine how to handle partitioned topics. The current method already calls `grantPermissionAsync` with that `topicName`, so we might be able to simply deprecate passing each partition to the `AuthorizationProvider`.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                        grantPermissionsAsync(TopicName.get(topicName.getPartitionedTopicName()), role, actions)
+                                .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()))))
+                .exceptionally(ex -> {
                     Throwable realCause = FutureUtil.unwrapCompletionException(ex);
                     log.error("[{}] Failed to get permissions for topic {}", clientAppId(), topicName, realCause);
                     resumeAsyncResponseExceptionally(asyncResponse, realCause);
                     return null;
                 });
     }
 
-    private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String role) {
+    private CompletableFuture<Void> revokePermissionsAsync(TopicName topicName, String role, int numPartitions) {
+        String topicUri = topicName.toString();
         return namespaceResources().getPoliciesAsync(namespaceName).thenCompose(
                 policiesOptional -> {
                     Policies policies = policiesOptional.orElseThrow(() ->
                             new RestException(Status.NOT_FOUND, "Namespace does not exist"));
+                    CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+                    // do compatible with previous pulsar version
+                    // revoke all the partition permissions granted in previous version
+                    future = future.thenComposeAsync(unused ->
+                        namespaceResources().setPoliciesAsync(namespaceName, p -> {
+                            if (numPartitions > 0) {
+                                for (int i = 0; i < numPartitions; i++) {
+                                    p.auth_policies.getTopicAuthentication().computeIfPresent(
+                                            topicName.getPartition(i).toString(), (k, roles) -> {
+                                                roles.remove(role);
+                                                if (roles.isEmpty()) {
+                                                    return null;
+                                                }
+                                                return roles;
+                                            });
+                                }
+                            }
+                            return p;
+                        }));
                     if (!policies.auth_policies.getTopicAuthentication().containsKey(topicUri)
                             || !policies.auth_policies.getTopicAuthentication().get(topicUri).containsKey(role)) {

Review Comment:
   I wonder if instead of failing here, we should just attempt to remove permission on every partition, even if it isn't in the map? I think that would be the expected behavior. This would also be backwards compatible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1235029060

   > > This PR remain as the second PR to optimize the implementation of permission. I am not familiar with how to get consensus, so I would wait for update this pr until the implementation is approved.
   > 
   > You can do this simply by sending an email to the [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) email indicating the change you'd like to make. If you are unable to do this, I can send the email.
   
   Thanks for your guide, I have sent an email to dev@pulsar.apache.org.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r962444260


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                        grantPermissionsAsync(TopicName.get(topicName.getPartitionedTopicName()), role, actions)
+                                .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()))))
+                .exceptionally(ex -> {
                     Throwable realCause = FutureUtil.unwrapCompletionException(ex);
                     log.error("[{}] Failed to get permissions for topic {}", clientAppId(), topicName, realCause);
                     resumeAsyncResponseExceptionally(asyncResponse, realCause);
                     return null;
                 });
     }
 
-    private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String role) {
+    private CompletableFuture<Void> revokePermissionsAsync(TopicName topicName, String role, int numPartitions) {
+        String topicUri = topicName.toString();
         return namespaceResources().getPoliciesAsync(namespaceName).thenCompose(
                 policiesOptional -> {
                     Policies policies = policiesOptional.orElseThrow(() ->
                             new RestException(Status.NOT_FOUND, "Namespace does not exist"));
+                    CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+                    // do compatible with previous pulsar version
+                    // revoke all the partition permissions granted in previous version
+                    future = future.thenComposeAsync(unused ->
+                        namespaceResources().setPoliciesAsync(namespaceName, p -> {
+                            if (numPartitions > 0) {
+                                for (int i = 0; i < numPartitions; i++) {
+                                    p.auth_policies.getTopicAuthentication().computeIfPresent(
+                                            topicName.getPartition(i).toString(), (k, roles) -> {
+                                                roles.remove(role);
+                                                if (roles.isEmpty()) {
+                                                    return null;
+                                                }
+                                                return roles;
+                                            });
+                                }
+                            }
+                            return p;
+                        }));
                     if (!policies.auth_policies.getTopicAuthentication().containsKey(topicUri)
                             || !policies.auth_policies.getTopicAuthentication().get(topicUri).containsKey(role)) {

Review Comment:
   Fixed by #17393.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1236502995

   @TakaHiR07 Any updates?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r996724004


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -273,6 +273,24 @@ protected CompletableFuture<Map<String, Set<AuthAction>>> internalGetPermissions
                         }
                     }
                 }
+
+                // If topic is partitioned, add based topic permission

Review Comment:
   @michaeljmarshall Is your idea the same as I described in  https://github.com/apache/pulsar/pull/16792#issuecomment-1219114676 ?? That's what I thought at the beginning, marked as plan 1.
   
   plan 2 is described in https://github.com/apache/pulsar/pull/16792#issue-1317797371, this one is designed based on the idea of @nodece .
   
   Both of the two plan can enhance.  I think plan 1 would have better compatibility with previous version, while plan 2 would have cleaner permission record.
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] congbobo184 commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1318544615

   @TakaHiR07  hi, I move this PR to release/2.9.5, if you have any questions, please ping me. thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] poorbarcode commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by "poorbarcode (via GitHub)" <gi...@apache.org>.
poorbarcode commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1501548259

   Since we will start the RC version of `3.0.0` on `2023-04-11`, I will change the label/milestone of PR who have not been merged.
   - The PR of type `feature` is deferred to `3.1.0`
   - The PR of type `fix` is deferred to `3.0.1`
   
   So drag this PR to `3.0.1`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1373042049

   The pr had no activity for 30 days, mark with Stale label.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1233700247

   As you saw, the CI fails.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1236517891

   > @TakaHiR07 Any updates?
   
   Haven't updated. I would update the implementation in a few day.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r951134496


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -319,21 +319,10 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
                                                    Set<AuthAction> actions) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()
+                        .thenCompose(unused1 -> grantPermissionsAsync(topicName, role, actions)

Review Comment:
   Should be `topicName.getPartitionedTopicName()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1219123399

   > I have changed the proposal and I think it is the better way to solve this problem and meet the compatibility.
   > 
   > pulsar old version
   > 
   > partition (e.g., persistent://public/default/mytopic-partition-0)	based topic (e.g., persistent://public/default/mytopic)
   > grant	only partition	grant all partition and then grant based topic
   > revoke	only partition	revoke all partition and then revoke based topic (shutdown if throw exception)
   > get	only partition	only based topic
   > check	check partition and then check based topic	only based topic
   > pulsar new version
   > 
   > partition (e.g., persistent://public/default/mytopic-partition-0)	based topic (e.g., persistent://public/default/mytopic)
   > grant	only partition	only based topic
   > revoke	only partition	revoke based topic and then revoke partition
   > get	only partition	only based topic
   > check	check partition and then check based topic	only based topic
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937517149


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -319,21 +319,12 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
                                                    Set<AuthAction> actions) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
+                    CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+                    return future.thenCompose(unused ->

Review Comment:
   Then covert the topic name to basic topic name in `org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider#grantPermissionAsync()`.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937510637


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -368,22 +359,12 @@ private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String r
     protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-            getPartitionedTopicMetadataAsync(topicName, true, false)
-                .thenCompose(metadata -> {
-                    int numPartitions = metadata.partitions;
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
                     CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                    if (numPartitions > 0) {
-                        for (int i = 0; i < numPartitions; i++) {
-                            TopicName topicNamePartition = topicName.getPartition(i);
-                            future = future.thenComposeAsync(unused ->
-                                    revokePermissionsAsync(topicNamePartition.toString(), role));
-                        }
-                    }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return future.thenComposeAsync(unused ->

Review Comment:
   You also need to check whether the topic is partitioned topic, then remove the permissions of each partition topic.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by "TakaHiR07 (via GitHub)" <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1584037362

   > @TakaHiR07 - I am so sorry for losing track of this PR. Are you interested in continuing your work on it? If not, I am going to pick up the work in a couple weeks. Thank you for your valuable observations and contributions so far!
   
   There is another pr about this work, you can take a look of it. https://github.com/apache/pulsar/pull/18222. I am glad to continue improve this work


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] michaeljmarshall commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1234407056

   > This PR remain as the second PR to optimize the implementation of permission. I am not familiar with how to get consensus, so I would wait for update this pr until the implementation is approved.
   
   You can do this simply by sending an email to the dev@pulsar.apache.org email indicating the change you'd like to make. If you are unable to do this, I can send the email.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1244961721

   The code has been updated. PTAL. @michaeljmarshall @nodece 
   With #17393, we can simply change the grantPermission() at AuthorizationProvider, and internalGetPermissionsOnTopic() at PersistentTopicsBase


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 closed pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 closed pull request #16792: fix can not revoke permission after update topic partition
URL: https://github.com/apache/pulsar/pull/16792


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r951135162


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -380,7 +369,7 @@ protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, Str
                                     revokePermissionsAsync(topicNamePartition.toString(), role));
                         }
                     }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return revokePermissionsAsync(topicName.toString(), role)

Review Comment:
   Should be `topicName.getPartitionedTopicName()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1222005341

   This table looks confused, I think you should follow the https://github.com/apache/pulsar/pull/16792?notification_referrer_id=NT_kwDOAPe6cbM0MDcwMTQyMTMyOjE2MjM1MTIx&notifications_query=repo%3Aapache%2Fpulsar#discussion_r938428577, you also need to improve the get action.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r938428577


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   @TakaHiR07 There are five scenarios:
   
   1. Grant permission
   Based on `topicName.getPartitionedTopicName()`
   
   2. Revoke permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic 
   
   4. Get permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic 
   
   5. Check permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic 
   
   Using the above rule can meet the old and new versions, no matter how the user grants permission.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r938446392


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   OK, I would update this pr soon. Thanks a lot



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r962481058


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -271,6 +271,24 @@ protected CompletableFuture<Map<String, Set<AuthAction>>> internalGetPermissions
                         }
                     }
                 }
+
+                // If topic is partitioned, add based topic permission
+                if (topicName.isPartitioned() && auth.getTopicAuthentication().containsKey(
+                        topicName.getPartitionedTopicName())) {
+                    for (Map.Entry<String, Set<AuthAction>> entry :
+                            auth.getTopicAuthentication().get(topicName.getPartitionedTopicName()).entrySet()) {
+                        String role = entry.getKey();
+                        Set<AuthAction> topicPermissions = entry.getValue();
+
+                        if (!permissions.containsKey(role)) {
+                            permissions.put(role, topicPermissions);
+                        } else {
+                            // Do the union between namespace and topic level
+                            Set<AuthAction> union = Sets.union(permissions.get(role), topicPermissions);
+                            permissions.put(role, union);
+                        }
+                    }
+                }

Review Comment:
   > The implementation of grant is in AuthorizationProvider, while get/revoke implementation is in persistentTopicsBase.
   Yes.
   
   > Is it better to provide the get/revoke permission method in AuthorizationProvider?
   
   Just is a suggestion, you don't need to do any updates.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] michaeljmarshall commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by "michaeljmarshall (via GitHub)" <gi...@apache.org>.
michaeljmarshall commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1584022545

   @TakaHiR07 - I am so sorry for losing track of this PR. Are you interested in continuing your work on it? If not, I am going to pick up the work in a couple weeks. Thank you for your valuable observations and contributions so far!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937450225


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -319,21 +319,12 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
                                                    Set<AuthAction> actions) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }
-                      return future.thenCompose(unused -> grantPermissionsAsync(topicName, role, actions))
-                              .thenAccept(unused -> asyncResponse.resume(Response.noContent().build()));
-                  }))).exceptionally(ex -> {
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
+                    CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+                    return future.thenCompose(unused ->

Review Comment:
   ```suggestion
                       return grantPermissionsAsync(TopicName.get(topicName.getPartitionedTopicName()), role, actions)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r938428577


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   @TakaHiR07 There are five scenarios:
   
   1. Grant permission
   Based on `topicName.getPartitionedTopicName()`
   
   2. Revoke permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic(my-topic-partition-0, my-topic-partition-1, ....)
   
   4. Get permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic(my-topic-partition-0, my-topic-partition-1, ....)
   
   5. Check permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic(my-topic-partition-0, my-topic-partition-1, ....)
   
   Using the above rule can meet the old and new versions, no matter how the user grants permission.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r938428577


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   @TakaHiR07 There are five scenarios:
   
   1. Grant permission
   Based on `topicName.getPartitionedTopicName()`
   
   2. Revoke permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic(my-topic-partition-0, my-topic-partition-1, ....)
   
   4. Get permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic(my-topic-partition-0, my-topic-partition-1, ....)
   
   5. Check permission
   Based on `topicName.getPartitionedTopicName()`, and full name of  each partition topic(my-topic-partition-0, my-topic-partition-1, ....)
   
   Using the above rule can meet the old and new versions, no matter how the user grants permission.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937622759


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   I mean that both can produce messages.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937564656


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   Do you mean that we should guarantee the compatibility for this situation?
   
   - someone only grant permission for persistent://test/test/test-partition-5, but not grant permission for persistent://test/test/test
   - he should still can produce msg to persistent://test/test/test-partition-5, but can not produce msg to persistent://test/test/test
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1221950018

   /pulsarbot rerun-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r962444572


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }

Review Comment:
   Agreed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] michaeljmarshall commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r996613942


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -324,32 +342,41 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
                 .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-             getPartitionedTopicMetadataAsync(topicName, true, false)
-                  .thenCompose(metadata -> {
-                      int numPartitions = metadata.partitions;
-                      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                      if (numPartitions > 0) {
-                          for (int i = 0; i < numPartitions; i++) {
-                              TopicName topicNamePartition = topicName.getPartition(i);
-                              future = future.thenCompose(unused -> grantPermissionsAsync(topicNamePartition, role,
-                                      actions));
-                          }
-                      }

Review Comment:
   This is all that is required for the optimization (regarding my other comment).



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -273,6 +273,24 @@ protected CompletableFuture<Map<String, Set<AuthAction>>> internalGetPermissions
                         }
                     }
                 }
+
+                // If topic is partitioned, add based topic permission

Review Comment:
   I do not think we should grant permission based on the partition's base topic. If the call is about a specific permission, we should respond with the result for that partition.



##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -247,7 +247,8 @@ public CompletableFuture<Void> grantPermissionAsync(TopicName topicName, Set<Aut
                 }
                 throw new IllegalStateException("policies are in readonly mode");
             }
-            String topicUri = topicName.toString();
+            // Enhancement: only grant permission on based topic
+            String topicUri = topicName.getPartitionedTopicName();

Review Comment:
   I think this might have been a misunderstanding of my review. The optimization is simply that we do not recursively add topic names when granting permission in the `PersistentTopicsBase#internalGrantPermissionsOnTopic` method. I think it is valid to grant a role permission to produce to a single partition of a partitioned topic, so we shouldn't make this update here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 closed pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by "TakaHiR07 (via GitHub)" <gi...@apache.org>.
TakaHiR07 closed pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata
URL: https://github.com/apache/pulsar/pull/16792


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1233971743

   > Thanks for your contribution @TakaHiR07. This is a great start. However, I think we'll need some additional work before it is ready to be merged.
   > 
   > First, I think this work will need to be two PRs.
   > 
   > There will be a PR to update how permissions are revoked. Essentially, we should not fail a request that attempts to remove permission from a partition of a topic that was not already granted permission. We will need to determine how to handle the case that a non-partitioned topic or all of the partitions of a partitioned topic (including the partitioned topic itself) does not have permission. This PR will be backwards compatible and will therefore be able to be cherry picked to existing release branches.
   > 
   > Then there is a PR that will change what the `PersistentTopicsBase` passes to the `AuthorizationService`. This PR is an optimization to limit the amount of metadata stored in ZK. Instead of passing each topic partition to the `grantPermissionsAsync` method, we'll just pass the topic that the user is attempting to grant permission on. This will technically be a breaking change for custom implemenations of the `AuthorizationProvider` interface, so we cannot cherry pick this change.
   > 
   > In order to be upgrade and downgrade compatible, we'll need to leave the revoke permission logic the same for a multiple minor versions. This work could be tracked in an issue.
   > 
   > Since both PRs will have to do with authorization and what the Pulsar Broker passes to the pluggable `AuthorizationProvider`, I think we should give a notice on the [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) mailing list to get consensus. I expect that this will not be a blocker for the feature/bug fix.
   
   I agree with it. I open another PR to update how permissions are revoked. 
   
   This PR remain as the second PR to optimize  the implementation of permission. I am not familiar with how to get consensus, so I would wait for update this pr until the implementation is approved. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1233972212

   > Thanks for your contribution @TakaHiR07. This is a great start. However, I think we'll need some additional work before it is ready to be merged.
   > 
   > First, I think this work will need to be two PRs.
   > 
   > There will be a PR to update how permissions are revoked. Essentially, we should not fail a request that attempts to remove permission from a partition of a topic that was not already granted permission. We will need to determine how to handle the case that a non-partitioned topic or all of the partitions of a partitioned topic (including the partitioned topic itself) does not have permission. This PR will be backwards compatible and will therefore be able to be cherry picked to existing release branches.
   > 
   > Then there is a PR that will change what the `PersistentTopicsBase` passes to the `AuthorizationService`. This PR is an optimization to limit the amount of metadata stored in ZK. Instead of passing each topic partition to the `grantPermissionsAsync` method, we'll just pass the topic that the user is attempting to grant permission on. This will technically be a breaking change for custom implemenations of the `AuthorizationProvider` interface, so we cannot cherry pick this change.
   > 
   > In order to be upgrade and downgrade compatible, we'll need to leave the revoke permission logic the same for a multiple minor versions. This work could be tracked in an issue.
   > 
   > Since both PRs will have to do with authorization and what the Pulsar Broker passes to the pluggable `AuthorizationProvider`, I think we should give a notice on the [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) mailing list to get consensus. I expect that this will not be a blocker for the feature/bug fix.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1233713219

   > As you saw, the CI fails.
   
   @nodece As you suggest, I have completely change the implementation of grant/revoke/get/check permission. Therefore many relevant unittest become flaky. 
   
   What do you think of this new implementation? If there are no other problems, I would fix the relevant flaky test.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937507456


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   You cannot remove this code, we need to do compatibility here. For the old version, we grant the permissions for each partitioned topic name, right now we only grant the permissions for the base partitioned topic name.
   
   You should check whether the topic is a partitioned topic. If true, we need to check the permissions by the full name of each partition topic.
   
   Assume the `persistent://my-tenant/my-namespace/my-topic` is partitioned topic, and has two partition:
   
   - `persistent://my-tenant/my-namespace/my-topic-partition-0`
   - `persistent://my-tenant/my-namespace/my-topic-partition-1`
   
   You first need to get the full name of each partition topic, then check the permissions of these topic names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937449050


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -368,22 +359,12 @@ private CompletableFuture<Void> revokePermissionsAsync(String topicUri, String r
     protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) {
         // This operation should be reading from zookeeper and it should be allowed without having admin privileges
         validateAdminAccessForTenantAsync(namespaceName.getTenant())
-                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 ->
-            getPartitionedTopicMetadataAsync(topicName, true, false)
-                .thenCompose(metadata -> {
-                    int numPartitions = metadata.partitions;
+                .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> {
                     CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
-                    if (numPartitions > 0) {
-                        for (int i = 0; i < numPartitions; i++) {
-                            TopicName topicNamePartition = topicName.getPartition(i);
-                            future = future.thenComposeAsync(unused ->
-                                    revokePermissionsAsync(topicNamePartition.toString(), role));
-                        }
-                    }
-                    return future.thenComposeAsync(unused -> revokePermissionsAsync(topicName.toString(), role))
+                    return future.thenComposeAsync(unused ->

Review Comment:
   ```suggestion
                       return revokePermissionsAsync(topicName, role)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937492638


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -370,7 +370,7 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                         }
 
                         Map<String, Set<AuthAction>> topicRoles = policies.get().auth_policies.getTopicAuthentication()
-                                .get(topicName.toString());
+                                .get(topicName.getPartitionedTopicName());

Review Comment:
   Revert this.
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] TakaHiR07 commented on a diff in pull request #16792: fix can not revoke permission after update topic partition

Posted by GitBox <gi...@apache.org>.
TakaHiR07 commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r937564656


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -392,23 +392,6 @@ public CompletableFuture<Boolean> checkPermission(TopicName topicName, String ro
                                 return true;
                             }
                         }
-

Review Comment:
   Do you mean that we should guarantee the compatibility for this situation?
   
   - someone only grant permission for persistent://test/test/test-partition-5, but not grant permission for persistent://test/test/test. he should still can produce msg to persistent://test/test/test-partition-5, but can not produce msg to persistent://test/test/test
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#issuecomment-1278382852

   The pr had no activity for 30 days, mark with Stale label.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] nodece commented on a diff in pull request #16792: [improve][broker] enhance grant permission module to reduce zk metadata

Posted by GitBox <gi...@apache.org>.
nodece commented on code in PR #16792:
URL: https://github.com/apache/pulsar/pull/16792#discussion_r996635694


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -247,7 +247,8 @@ public CompletableFuture<Void> grantPermissionAsync(TopicName topicName, Set<Aut
                 }
                 throw new IllegalStateException("policies are in readonly mode");
             }
-            String topicUri = topicName.toString();
+            // Enhancement: only grant permission on based topic
+            String topicUri = topicName.getPartitionedTopicName();

Review Comment:
   Good catch! All partitions should have the same permissions, is that right?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org