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/09/01 17:29:21 UTC

[GitHub] [pulsar] KannarFr opened a new pull request, #17411: Check create topic on subscription creation

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

   Fixes #17406 
   
   ### Motivation
   
   Enable CREATE_TOPIC permission check on topic auto creation.
   
   ### Modifications
   
   Introduce AuthAction.create_topic for the PulsarAuthorizationProvider
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   This change updated tests and can be verified as follows:
   
   - updated test and give create_topic permission on the namespace
   
   ### Does this pull request potentially affect one of the following parts:
   
   - it affects permissions as currently, CREATE_TOPIC wasn't checked on the default authZ provider.
   


-- 
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] MarvinCai commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -482,12 +482,12 @@ public CompletableFuture<Boolean> allowNamespaceOperationAsync(NamespaceName nam
                             case GET_TOPIC:
                             case GET_TOPICS:
                             case GET_BUNDLE:
+                            case CREATE_TOPIC:
                                 return allowConsumeOrProduceOpsAsync(namespaceName, role, authData);

Review Comment:
   seems we're only checking permission for `consume` or `produce` action here, I'm wondering what's the benefit of this pr while we don't have a action for `create_topic` itself



-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

Posted by "michaeljmarshall (via GitHub)" <gi...@apache.org>.
michaeljmarshall commented on code in PR #17411:
URL: https://github.com/apache/pulsar/pull/17411#discussion_r1170845565


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1109,13 +1109,18 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 TopicOperation.CONSUME
         );
 
+        CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                topicName.getNamespaceObject(),
+                NamespaceOperation.CREATE_TOPIC
+        );

Review Comment:
   As I mentioned before, this does not look backwards/forwards compatible. I don't think we can add required `AuthActions` without finding a way to make them compatible. The idea is that users should be able to upgrade then downgrade and things should still work. Can you confirm @KannarFr?



-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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

   > > Perhaps we can introduce an additional check that calls the authorization service and then we'll implement the default PulsarAuthorizationProvider in such a way that there are no breaking change
   > 
   > This LGTM. Can you drive this discussion on the ML if you think it's required? Shall I do it?
   
   I won't be able to drive this discussion. Let me know if you have any questions though.


-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -2361,8 +2361,11 @@ private void maybeOffloadInBackground(CompletableFuture<PositionImpl> promise) {
         if (config.getLedgerOffloader() != null
                 && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
                 && config.getLedgerOffloader().getOffloadPolicies() != null
-                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() != null
-                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0) {
+                && ((config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() != null
+                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0)
+                || (config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInMillis() != null
+                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInMillis() >= 0))

Review Comment:
   I think the changes made in this file might belong in a separate PR. Would you mind making a separate PR for them?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));

Review Comment:
   I think we want the following in order to retain the existing logic:
   
   ```suggestion
                   boolean createTopicIfDoesNotExist = isAuthorizedToCreateTopic || (forceTopicCreation
                           && service.isAllowAutoTopicCreation(topicName.toString()));
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1307,7 +1326,25 @@ protected void handleProducer(final CommandProducer cmdProducer) {
                         producerId, schema == null ? "absent" : "present");
             }
 
-            service.getOrCreateTopic(topicName.toString()).thenCompose((Topic topic) -> {
+            boolean createTopicIfDoesNotExist = isAuthorizedToCreateTopic
+                    && service.isAllowAutoTopicCreation(topicName.toString());
+            service.getTopic(topicName.toString(), createTopicIfDoesNotExist).thenCompose(optTopic -> {
+                if (optTopic.isEmpty()) {
+                    if (isAuthorizedToCreateTopic) {
+                        return FutureUtil
+                                .failedFuture(new TopicNotFoundException(
+                                        "Topic " + topicName + " does not exist"));
+                    } else {
+                        String msg = "Topic to produce does not exists and the Client is not"
+                                + " authorized to create topic";
+                        log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                        ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                msg));
+                        return null;

Review Comment:
   Similarly, we need to throw an exception here. The error handling includes handling the dangling `producer` future, and we need to do that to prevent a build up of unnecessary futures.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));
                 service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                         .thenCompose(optTopic -> {
                             if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
+                                if (isAuthorizedToCreateTopic) {
+                                    return FutureUtil
+                                            .failedFuture(new TopicNotFoundException(
+                                                    "Topic " + topicName + " does not exist"));
+                                } else {
+                                    String msg = "Topic to subscribe does not exists and the Client is not"
+                                            + " authorized to create topic";
+                                    log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                                    consumers.remove(consumerId, consumerFuture);
+                                    ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                            msg));
+                                    return null;

Review Comment:
   I don't think returning `null` here will give the desired behavior unless we modify the `thenAccept` block. It seems like the current paradigm in this section of the code is to throw an exception, so I think it would be appropriate to do that 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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

Posted by "KannarFr (via GitHub)" <gi...@apache.org>.
KannarFr commented on code in PR #17411:
URL: https://github.com/apache/pulsar/pull/17411#discussion_r1161935338


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));
                 service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                         .thenCompose(optTopic -> {
                             if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
+                                if (isAuthorizedToCreateTopic) {
+                                    return FutureUtil
+                                            .failedFuture(new TopicNotFoundException(
+                                                    "Topic " + topicName + " does not exist"));
+                                } else {
+                                    String msg = "Topic to subscribe does not exists and the Client is not"
+                                            + " authorized to create topic";
+                                    log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                                    consumers.remove(consumerId, consumerFuture);
+                                    ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                            msg));
+                                    return null;

Review Comment:
   I just rebased it from master.



-- 
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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   @michaeljmarshall are you ok with the way I've done it in handleProducer?


-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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

   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] michaeljmarshall commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,12 +1082,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = isAuthorizedToCreateTopic && forceTopicCreation
+                        && service.isAllowAutoTopicCreation(topicName.toString());

Review Comment:
   By making each of these required, it breaks the current contract, which is that permission to produce to or consume from a topic is sufficient to auto create a topic when auto topic creation is enabled.
   
   I think it would make more sense to add this feature as an alternative to auto topic creation since we're explicitly giving a producer or consumer permission to create a topic. I think changing the PR to align with my suggestion is as simple as using `or` instead of `and` in this part of the code and any other relevant parts:
   
   ```suggestion
                   boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
                           && service.isAllowAutoTopicCreation(topicName.toString()));
   ```
   
   Note that the `forceTopicCreation` is from the consumer, and I think we should honor that input even when the consumer has permission to create the 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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,12 +1082,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = isAuthorizedToCreateTopic && forceTopicCreation
+                        && service.isAllowAutoTopicCreation(topicName.toString());

Review Comment:
   LGTM.



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -2361,8 +2361,11 @@ private void maybeOffloadInBackground(CompletableFuture<PositionImpl> promise) {
         if (config.getLedgerOffloader() != null
                 && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
                 && config.getLedgerOffloader().getOffloadPolicies() != null
-                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() != null
-                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0) {
+                && ((config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() != null
+                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0)
+                || (config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInMillis() != null
+                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInMillis() >= 0))

Review Comment:
   Wow clearly. My bad.



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -482,12 +482,12 @@ public CompletableFuture<Boolean> allowNamespaceOperationAsync(NamespaceName nam
                             case GET_TOPIC:
                             case GET_TOPICS:
                             case GET_BUNDLE:
+                            case CREATE_TOPIC:
                                 return allowConsumeOrProduceOpsAsync(namespaceName, role, authData);

Review Comment:
   It is not solving the issue. It is required to make the test pass to solve the issue. I could create a create_topic AuthAction, but it won't be backward compatible. WDYT?



-- 
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


Re: [PR] Check create topic permission on topic creation using pulsar proto clients [pulsar]

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

   Is there any way to move forward with this  PR ?
   
   publishing/consuming and creating/deleting a topic are clearly different actions, and thus should require different authorization. 
   
   Why would there be an api to create/delete topics if that was not the 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] Nowadays commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   Is there any progression on this issue ?


-- 
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 a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

Posted by "poorbarcode (via GitHub)" <gi...@apache.org>.
poorbarcode commented on code in PR #17411:
URL: https://github.com/apache/pulsar/pull/17411#discussion_r1161895207


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));
                 service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                         .thenCompose(optTopic -> {
                             if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
+                                if (isAuthorizedToCreateTopic) {
+                                    return FutureUtil
+                                            .failedFuture(new TopicNotFoundException(
+                                                    "Topic " + topicName + " does not exist"));
+                                } else {
+                                    String msg = "Topic to subscribe does not exists and the Client is not"
+                                            + " authorized to create topic";
+                                    log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                                    consumers.remove(consumerId, consumerFuture);
+                                    ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                            msg));
+                                    return null;

Review Comment:
   Hi @michaeljmarshall 
   
   Could you retake a look? 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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   > After reviewing this, here are a collection of thoughts on Pulsar's Authorization.
   > 
   > At it's core, I think the problem described by #17406 is that `allowAutoTopicCreation` is a configuration about permission/authorization.
   
   Yes?
   
   > In my view, the `allowAutoTopicCreation=true` says "a role with permission to produce/consume to a topic also has permission to create that topic".
   
   The HTTP admin API does not comply with this sentence. The CREATE_TOPIC operation is defined and used by HTTP admin API authz checks. I agree that this is introducing breaking changes in the permissions system and this is a problem, but there is authZ plugin provider providing this operation check and does not verify it during producer/consumer. So, I have no idea what the best answer is, but we can't stay and need to find a solution or make a decision here.
   
   > This change proposes that `allowAutoTopicCreation=true` and produce/consumer permissions are insufficient, and that a role must also have explicit permission to create a topic.
   
   Well, this is the default behavior of every configurations keys in pulsar. Like namespace policies or tiered storage configurations, there is a default value that can be overridden by custom namespace policies.
   
   
   
   


-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

Posted by "KannarFr (via GitHub)" <gi...@apache.org>.
KannarFr commented on code in PR #17411:
URL: https://github.com/apache/pulsar/pull/17411#discussion_r1171121469


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1109,13 +1109,18 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 TopicOperation.CONSUME
         );
 
+        CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                topicName.getNamespaceObject(),
+                NamespaceOperation.CREATE_TOPIC
+        );

Review Comment:
   That would be nice, but as I answered you am I not sure we can really do it has CREATE_TOPIC check is already performed in HTTP admin checks.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1109,13 +1109,18 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 TopicOperation.CONSUME
         );
 
+        CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                topicName.getNamespaceObject(),
+                NamespaceOperation.CREATE_TOPIC
+        );

Review Comment:
   That would be nice, but as I answered you am I not sure we can really do it as CREATE_TOPIC check is already performed in HTTP admin 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] michaeljmarshall commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

Posted by "michaeljmarshall (via GitHub)" <gi...@apache.org>.
michaeljmarshall commented on code in PR #17411:
URL: https://github.com/apache/pulsar/pull/17411#discussion_r1171527978


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1109,13 +1109,18 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 TopicOperation.CONSUME
         );
 
+        CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                topicName.getNamespaceObject(),
+                NamespaceOperation.CREATE_TOPIC
+        );

Review Comment:
   While I agree that the current design is inconsistent, I don't think we should add logic that breaks existing use cases.



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -482,12 +482,12 @@ public CompletableFuture<Boolean> allowNamespaceOperationAsync(NamespaceName nam
                             case GET_TOPIC:
                             case GET_TOPICS:
                             case GET_BUNDLE:
+                            case CREATE_TOPIC:
                                 return allowConsumeOrProduceOpsAsync(namespaceName, role, authData);

Review Comment:
   The default authZ provider hasn't but it AuthorizationProvider is a plugin interface that we are using with biscuit token: https://github.com/CleverCloud/biscuit-pulsar.



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,102 +1077,126 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
-                service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
-                        .thenCompose(optTopic -> {
-                            if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
-                            }
 
-                            Topic topic = optTopic.get();
+                CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                        topicName.getNamespaceObject(),
+                        NamespaceOperation.CREATE_TOPIC

Review Comment:
   This will prevent produce/consume of topic if it already exist but create_topic operation not authorized, nope?



-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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

   @KannarFr Please add the following content to your PR description and select a checkbox:
   ```
   - [ ] `doc` <!-- Your PR contains doc changes -->
   - [ ] `doc-required` <!-- Your PR changes impact docs and you will update later -->
   - [ ] `doc-not-needed` <!-- Your PR changes do not impact docs -->
   - [ ] `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


Re: [PR] Check create topic permission on topic creation using pulsar proto clients [pulsar]

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

   @michaeljmarshall @gaoran10 @MarvinCai, it makes sense to rebase 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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,102 +1077,126 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
-                service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
-                        .thenCompose(optTopic -> {
-                            if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
-                            }
 
-                            Topic topic = optTopic.get();
+                CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                        topicName.getNamespaceObject(),
+                        NamespaceOperation.CREATE_TOPIC

Review Comment:
   This will prevent produce/consume of topic if it already exists but create_topic operation not authorized, nope?



-- 
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] gaoran10 commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,102 +1077,126 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
-                service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
-                        .thenCompose(optTopic -> {
-                            if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
-                            }
 
-                            Topic topic = optTopic.get();
+                CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                        topicName.getNamespaceObject(),
+                        NamespaceOperation.CREATE_TOPIC

Review Comment:
   Maybe we can add this before [line 1026](https://github.com/apache/pulsar/pull/17411/files#diff-1e0e8195fb5ec5a6d79acbc7d859c025a9b711f94e6ab37c94439e99b3202e84L1026), and use this method `isAuthorizedFuture.thenCombine(isAuthorizedToCreateTopicFuture, (isAuthorized, isAuthorizedToCreateTopic)`.



-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java:
##########
@@ -2361,8 +2361,11 @@ private void maybeOffloadInBackground(CompletableFuture<PositionImpl> promise) {
         if (config.getLedgerOffloader() != null
                 && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
                 && config.getLedgerOffloader().getOffloadPolicies() != null
-                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() != null
-                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0) {
+                && ((config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() != null
+                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0)
+                || (config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInMillis() != null
+                && config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInMillis() >= 0))

Review Comment:
   No problem.



-- 
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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   > Perhaps we can introduce an additional check that calls the authorization service and then we'll implement the default PulsarAuthorizationProvider in such a way that there are no breaking change
   
   This LGTM. Can you drive this discussion on the ML? Shall I do 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


Re: [PR] Check create topic permission on topic creation using pulsar proto clients [pulsar]

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

   Any bumps?


-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));
                 service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                         .thenCompose(optTopic -> {
                             if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
+                                if (isAuthorizedToCreateTopic) {
+                                    return FutureUtil
+                                            .failedFuture(new TopicNotFoundException(
+                                                    "Topic " + topicName + " does not exist"));
+                                } else {
+                                    String msg = "Topic to subscribe does not exists and the Client is not"
+                                            + " authorized to create topic";
+                                    log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                                    consumers.remove(consumerId, consumerFuture);
+                                    ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                            msg));
+                                    return null;

Review Comment:
   If we return `null` here, we will go to the `thenAccept` block and will call `consumerFuture.complete(consumer)`. We don't want to complete the future there. I think we wnat to run the lines 1187 to 1198 that you reference. Returning a failed future ensures that we jump to the `exceptionally` block on 1169 that will send the error response.



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));
                 service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                         .thenCompose(optTopic -> {
                             if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
+                                if (isAuthorizedToCreateTopic) {
+                                    return FutureUtil
+                                            .failedFuture(new TopicNotFoundException(
+                                                    "Topic " + topicName + " does not exist"));
+                                } else {
+                                    String msg = "Topic to subscribe does not exists and the Client is not"
+                                            + " authorized to create topic";
+                                    log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                                    consumers.remove(consumerId, consumerFuture);
+                                    ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                            msg));
+                                    return null;

Review Comment:
   Ok, does e34524abd0a9cea41e7968f518e6929647e4e140 LGTY?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1307,7 +1326,25 @@ protected void handleProducer(final CommandProducer cmdProducer) {
                         producerId, schema == null ? "absent" : "present");
             }
 
-            service.getOrCreateTopic(topicName.toString()).thenCompose((Topic topic) -> {
+            boolean createTopicIfDoesNotExist = isAuthorizedToCreateTopic
+                    && service.isAllowAutoTopicCreation(topicName.toString());
+            service.getTopic(topicName.toString(), createTopicIfDoesNotExist).thenCompose(optTopic -> {
+                if (optTopic.isEmpty()) {
+                    if (isAuthorizedToCreateTopic) {
+                        return FutureUtil
+                                .failedFuture(new TopicNotFoundException(
+                                        "Topic " + topicName + " does not exist"));
+                    } else {
+                        String msg = "Topic to produce does not exists and the Client is not"
+                                + " authorized to create topic";
+                        log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                        ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                msg));
+                        return null;

Review Comment:
   See previous thread.



-- 
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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   @gaoran10 the failing tests are due to CI flakyness, nope?


-- 
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] MarvinCai commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -482,12 +482,12 @@ public CompletableFuture<Boolean> allowNamespaceOperationAsync(NamespaceName nam
                             case GET_TOPIC:
                             case GET_TOPICS:
                             case GET_BUNDLE:
+                            case CREATE_TOPIC:
                                 return allowConsumeOrProduceOpsAsync(namespaceName, role, authData);

Review Comment:
   I see AuthorizationProviderBiscuit is check for namespace operator in `allowNamespaceOperationAsync`, but I don't see how moving up the `case CREATE_TOPIC:` in PulsarAuthorizationProvider is able solving the issue you described?
   A client that subscribe to a topic that does not exist will still able to create the topic cause it has consume 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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   @poorbarcode, @michaeljmarshall looks not available, can you ping someone else? It would be cool to merge it upstream asap :D.


-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

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

   > The HTTP admin API does not comply with this sentence.
   
   Fair point. I hadn't considered the pulsar and http endpoints together when writing that generalization.
   
   > I agree that this is introducing breaking changes in the permissions system and this is a problem, but there is authZ plugin provider providing this operation check and does not verify it during producer/consumer. 
   
   Is there another way to achieve this? Perhaps we can introduce an additional check that calls the authorization service and then we'll implement the default PulsarAuthorizationProvider in such a way that there are no breaking changes.
   
   I think it might make sense to discuss this inconsistency on the pulsar mailing list.


-- 
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 #17411: Check create topic permission on topic creation using pulsar proto clients

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #17411:
URL: https://github.com/apache/pulsar/pull/17411#issuecomment-1573031932

   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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   I sent
   
   > Hi,
   > 
   > CREATE_TOPIC authorization check is not performed when trying to PRODUCE/CONSUME a topic, it has been referenced: https://github.com/apache/pulsar/issues/17406.
   > 
   > I opened a PR to fix it https://github.com/apache/pulsar/pull/17411, but Michael reported issues about backward compatibility (which is totally correct). Adding support of CREATE_TOPIC authorization as-is will break current authorization system. I noticed that HTTP Admin API verifies the CREATE_TOPIC right when creating topic, so we have inconsistencies between pulsar binary protocol and the HTTP admin API on this.
   > 
   > Also, the AuthorizationProvider is an interface exposing the CREATE_TOPIC feature for authZ plugins. But it is inconsistent too.
   > 
   > Michael suggested to fix this interface to support the CREATE_TOPIC check and adapt the pulsar's DefaultAuthzProvider to continue as-is.
   > 
   > I'd like to know what do you think?
   > 
   > Thanks,
   > 
   > Kannar
   
   On the ML the 20/04/2023 and still have no answers @michaeljmarshall @nodece. 


-- 
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 #17411: Check create topic on subscription creation

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

   @KannarFr Please provide a correct documentation label for your PR.
   Instructions see [Pulsar Documentation Label Guide](https://docs.google.com/document/d/1Qw7LHQdXWBW9t2-r-A7QdFDBwmZh6ytB4guwMoXHqc0).


-- 
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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   @michaeljmarshall there is something I missed?


-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1089,12 +1091,25 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
+
+                boolean createTopicIfDoesNotExist = forceTopicCreation || (isAuthorizedToCreateTopic
+                        && service.isAllowAutoTopicCreation(topicName.toString()));
                 service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                         .thenCompose(optTopic -> {
                             if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
+                                if (isAuthorizedToCreateTopic) {
+                                    return FutureUtil
+                                            .failedFuture(new TopicNotFoundException(
+                                                    "Topic " + topicName + " does not exist"));
+                                } else {
+                                    String msg = "Topic to subscribe does not exists and the Client is not"
+                                            + " authorized to create topic";
+                                    log.warn("[{}] {} with role {}", remoteAddress, msg, getPrincipal());
+                                    consumers.remove(consumerId, consumerFuture);
+                                    ctx.writeAndFlush(Commands.newError(requestId, ServerError.AuthorizationError,
+                                            msg));
+                                    return null;

Review Comment:
   Did you see https://github.com/apache/pulsar/pull/17411/files#diff-1e0e8195fb5ec5a6d79acbc7d859c025a9b711f94e6ab37c94439e99b3202e84L1187-L1198 lines? It seems we are returning null and writing errors to the channel. Nope?



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,102 +1077,126 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
-                service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
-                        .thenCompose(optTopic -> {
-                            if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
-                            }
 
-                            Topic topic = optTopic.get();
+                CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                        topicName.getNamespaceObject(),
+                        NamespaceOperation.CREATE_TOPIC

Review Comment:
   Oh nvm, got 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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -1080,102 +1077,126 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
                 }
                 Optional<Map<String, String>> subscriptionProperties = SubscriptionOption.getPropertiesMap(
                         subscribe.getSubscriptionPropertiesList());
-                service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
-                        .thenCompose(optTopic -> {
-                            if (!optTopic.isPresent()) {
-                                return FutureUtil
-                                        .failedFuture(new TopicNotFoundException(
-                                                "Topic " + topicName + " does not exist"));
-                            }
 
-                            Topic topic = optTopic.get();
+                CompletableFuture<Boolean> isAuthorizedToCreateTopicFuture = isNamespaceOperationAllowed(
+                        topicName.getNamespaceObject(),
+                        NamespaceOperation.CREATE_TOPIC

Review Comment:
   @gaoran10 done.



-- 
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] KannarFr commented on a diff in pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -482,12 +482,12 @@ public CompletableFuture<Boolean> allowNamespaceOperationAsync(NamespaceName nam
                             case GET_TOPIC:
                             case GET_TOPICS:
                             case GET_BUNDLE:
+                            case CREATE_TOPIC:
                                 return allowConsumeOrProduceOpsAsync(namespaceName, role, authData);

Review Comment:
   It's a breaking change but I added your suggestion. WDYT?



##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java:
##########
@@ -482,12 +482,12 @@ public CompletableFuture<Boolean> allowNamespaceOperationAsync(NamespaceName nam
                             case GET_TOPIC:
                             case GET_TOPICS:
                             case GET_BUNDLE:
+                            case CREATE_TOPIC:
                                 return allowConsumeOrProduceOpsAsync(namespaceName, role, authData);

Review Comment:
   It is not solving the issue. It is required to make the test pass to solve the issue. I could create a create_topic AuthAction, but it won't be backward compatible. WDYT?



-- 
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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   Unfortunately, no. I don't know why this is stuck @michaeljmarshall?


-- 
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] KannarFr commented on pull request #17411: Check create topic permission on topic creation using pulsar proto clients

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

   @gaoran10 @MarvinCai 


-- 
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