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/18 07:32:42 UTC

[GitHub] [pulsar] nodece opened a new pull request, #16645: [fix][authorization] Fix multiple roles authorization

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

   Fixes #16574 
   
   ### Motivation
   
   The `MultiRolesTokenAuthorizationProvider` cannot handle the superuser and tenant admin correctly. It only checks if the role passed in is a super user, which is incorrect. The correct way is to get a role list from authentication data and then check if the role list contains a super user.
   
   ### Modifications
   
   - Override the `isSuperAdmin`
   - Override the `validateTenantAdminAccess`
   - Add test for the `MultiRolesTokenAuthorizationProvider`
   
   ### Documentation
   
   - [x] `doc-not-needed` 
   


-- 
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] Technoboy- commented on a diff in pull request #16645: [fix][authorization] Fix multiple roles authorization

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16645:
URL: https://github.com/apache/pulsar/pull/16645#discussion_r928598325


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java:
##########
@@ -80,58 +86,112 @@ public void initialize(ServiceConfiguration conf, PulsarResources pulsarResource
         super.initialize(conf, pulsarResources);
     }
 
-    private List<String> getRoles(AuthenticationDataSource authData) {
+    @Override
+    public CompletableFuture<Boolean> isSuperUser(String role, AuthenticationDataSource authenticationData,
+                                                  ServiceConfiguration serviceConfiguration) {
+        Set<String> roles = getRoles(authenticationData);
+        if (roles.isEmpty()) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Set<String> superUserRoles = serviceConfiguration.getSuperUserRoles();
+        if (superUserRoles.isEmpty()) {
+            return CompletableFuture.completedFuture(false);
+        }
+
+        return CompletableFuture.completedFuture(roles.stream().anyMatch(superUserRoles::contains));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> validateTenantAdminAccess(String tenantName, String role,
+                                                                AuthenticationDataSource authData) {
+        return isSuperUser(role, authData, conf)
+                .thenCompose(isSuperUser -> {
+                    if (isSuperUser) {
+                        return CompletableFuture.completedFuture(true);
+                    }
+                    Set<String> roles = getRoles(authData);
+                    if (roles.isEmpty()) {
+                        return CompletableFuture.completedFuture(false);
+                    }
+
+                    return pulsarResources.getTenantResources()
+                            .getTenantAsync(tenantName)
+                            .thenCompose(op -> {
+                                if (op.isPresent()) {
+                                    TenantInfo tenantInfo = op.get();
+                                    if (tenantInfo.getAdminRoles() == null || tenantInfo.getAdminRoles().isEmpty()) {
+                                        return CompletableFuture.completedFuture(false);
+                                    }
+
+                                    return CompletableFuture.completedFuture(roles.stream()
+                                            .anyMatch(n -> tenantInfo.getAdminRoles().contains(n)));
+                                } else {
+                                    throw new RestException(Response.Status.NOT_FOUND, "Tenant does not exist");
+                                }
+                            }).exceptionally(ex -> {
+                                Throwable cause = ex.getCause();
+                                if (cause instanceof MetadataStoreException.NotFoundException) {
+                                    log.warn("Failed to get tenant info data for non existing tenant {}", tenantName);
+                                    throw new RestException(Response.Status.NOT_FOUND, "Tenant does not exist");
+                                }
+                                log.error("Failed to get tenant {}", tenantName, cause);
+                                throw new RestException(cause);

Review Comment:
   If line 129 throws RestException, will here wrap to  new RestException(RestException ...) ?



-- 
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] RobertIndie commented on a diff in pull request #16645: [fix][authorization] Fix multiple roles authorization

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiRolesTokenAuthorizationProviderTest.java:
##########
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.api;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertThrows;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.Jwts;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.Cleanup;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.broker.authorization.MultiRolesTokenAuthorizationProvider;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class MultiRolesTokenAuthorizationProviderTest extends MockedPulsarServiceBaseTest {

Review Comment:
   Let's move this test to here: https://github.com/apache/pulsar/blob/master/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java



-- 
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 #16645: [fix][authorization] Fix multiple roles authorization

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiRolesTokenAuthorizationProviderTest.java:
##########
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.api;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertThrows;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.Jwts;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.Cleanup;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.broker.authorization.MultiRolesTokenAuthorizationProvider;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class MultiRolesTokenAuthorizationProviderTest extends MockedPulsarServiceBaseTest {

Review Comment:
   We cannot move this test to the pulsar-broker-common module, which is an integration test and depends on the pulsar-broker module.
   
   



-- 
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 #16645: [fix][authorization] Fix multiple roles authorization

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

   @BewareMyPower 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] Technoboy- commented on a diff in pull request #16645: [fix][authorization] Fix multiple roles authorization

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on code in PR #16645:
URL: https://github.com/apache/pulsar/pull/16645#discussion_r923085761


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java:
##########
@@ -80,58 +86,112 @@ public void initialize(ServiceConfiguration conf, PulsarResources pulsarResource
         super.initialize(conf, pulsarResources);
     }
 
-    private List<String> getRoles(AuthenticationDataSource authData) {
+    @Override
+    public CompletableFuture<Boolean> isSuperUser(String role, AuthenticationDataSource authenticationData,
+                                                  ServiceConfiguration serviceConfiguration) {
+        Set<String> roles = getRoles(authenticationData);
+        if (roles.isEmpty()) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Set<String> superUserRoles = serviceConfiguration.getSuperUserRoles();
+        if (superUserRoles.isEmpty()) {
+            CompletableFuture.completedFuture(false);

Review Comment:
   Miss `return `



-- 
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] RobertIndie commented on a diff in pull request #16645: [fix][authorization] Fix multiple roles authorization

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


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiRolesTokenAuthorizationProviderTest.java:
##########
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.api;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertThrows;
+import com.google.common.collect.Sets;
+import io.jsonwebtoken.Jwts;
+import io.jsonwebtoken.SignatureAlgorithm;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import javax.crypto.SecretKey;
+import lombok.Cleanup;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
+import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
+import org.apache.pulsar.broker.authorization.MultiRolesTokenAuthorizationProvider;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.impl.auth.AuthenticationToken;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class MultiRolesTokenAuthorizationProviderTest extends MockedPulsarServiceBaseTest {

Review Comment:
   Got it. 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] nodece commented on a diff in pull request #16645: [fix][authorization] Fix multiple roles authorization

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java:
##########
@@ -80,58 +86,112 @@ public void initialize(ServiceConfiguration conf, PulsarResources pulsarResource
         super.initialize(conf, pulsarResources);
     }
 
-    private List<String> getRoles(AuthenticationDataSource authData) {
+    @Override
+    public CompletableFuture<Boolean> isSuperUser(String role, AuthenticationDataSource authenticationData,
+                                                  ServiceConfiguration serviceConfiguration) {
+        Set<String> roles = getRoles(authenticationData);
+        if (roles.isEmpty()) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Set<String> superUserRoles = serviceConfiguration.getSuperUserRoles();
+        if (superUserRoles.isEmpty()) {
+            CompletableFuture.completedFuture(false);

Review Comment:
   Good catch.



-- 
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 #16645: [fix][authorization] Fix multiple roles authorization

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java:
##########
@@ -80,58 +86,112 @@ public void initialize(ServiceConfiguration conf, PulsarResources pulsarResource
         super.initialize(conf, pulsarResources);
     }
 
-    private List<String> getRoles(AuthenticationDataSource authData) {
+    @Override
+    public CompletableFuture<Boolean> isSuperUser(String role, AuthenticationDataSource authenticationData,
+                                                  ServiceConfiguration serviceConfiguration) {
+        Set<String> roles = getRoles(authenticationData);
+        if (roles.isEmpty()) {
+            return CompletableFuture.completedFuture(false);
+        }
+        Set<String> superUserRoles = serviceConfiguration.getSuperUserRoles();
+        if (superUserRoles.isEmpty()) {
+            return CompletableFuture.completedFuture(false);
+        }
+
+        return CompletableFuture.completedFuture(roles.stream().anyMatch(superUserRoles::contains));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> validateTenantAdminAccess(String tenantName, String role,
+                                                                AuthenticationDataSource authData) {
+        return isSuperUser(role, authData, conf)
+                .thenCompose(isSuperUser -> {
+                    if (isSuperUser) {
+                        return CompletableFuture.completedFuture(true);
+                    }
+                    Set<String> roles = getRoles(authData);
+                    if (roles.isEmpty()) {
+                        return CompletableFuture.completedFuture(false);
+                    }
+
+                    return pulsarResources.getTenantResources()
+                            .getTenantAsync(tenantName)
+                            .thenCompose(op -> {
+                                if (op.isPresent()) {
+                                    TenantInfo tenantInfo = op.get();
+                                    if (tenantInfo.getAdminRoles() == null || tenantInfo.getAdminRoles().isEmpty()) {
+                                        return CompletableFuture.completedFuture(false);
+                                    }
+
+                                    return CompletableFuture.completedFuture(roles.stream()
+                                            .anyMatch(n -> tenantInfo.getAdminRoles().contains(n)));
+                                } else {
+                                    throw new RestException(Response.Status.NOT_FOUND, "Tenant does not exist");
+                                }
+                            }).exceptionally(ex -> {
+                                Throwable cause = ex.getCause();
+                                if (cause instanceof MetadataStoreException.NotFoundException) {
+                                    log.warn("Failed to get tenant info data for non existing tenant {}", tenantName);
+                                    throw new RestException(Response.Status.NOT_FOUND, "Tenant does not exist");
+                                }
+                                log.error("Failed to get tenant {}", tenantName, cause);
+                                throw new RestException(cause);

Review Comment:
   RestException can get the real exception.
   
   ```java
   public RestException(Throwable t) {
           super(getResponse(t));
       }
   
       private static Response getResponse(Throwable t) {
           if (t instanceof WebApplicationException) {
               WebApplicationException e = (WebApplicationException) t;
               return e.getResponse();
           } else {
               return Response
                   .status(Status.INTERNAL_SERVER_ERROR)
                   .entity(getExceptionData(t))
                   .type(MediaType.TEXT_PLAIN)
                   .build();
           }
       }
   ```



-- 
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] Technoboy- merged pull request #16645: [fix][authorization] Fix multiple roles authorization

Posted by GitBox <gi...@apache.org>.
Technoboy- merged PR #16645:
URL: https://github.com/apache/pulsar/pull/16645


-- 
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] BewareMyPower commented on pull request #16645: [fix][authorization] Fix multiple roles authorization

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

   This PR relies on the `TenantResources#getTenantAsync` method, which relies on the `BaseResources#joinPath` method that I tried to migrate in https://github.com/apache/pulsar/pull/16867. Could you help review that PR so that I can continue cherry-picking and resolving the conflicts?


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