You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "Jackie-Jiang (via GitHub)" <gi...@apache.org> on 2023/07/21 06:34:44 UTC

[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #11016: Rbac impl

Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1270256005


##########
pinot-integration-tests/src/main/java/org/apache/pinot/broker/api/resources/BrokerEchoWithAutoDiscovery.java:
##########
@@ -40,6 +44,7 @@ public class BrokerEchoWithAutoDiscovery {
     public AutoLoadedServiceForTest _injectedService;
     @GET
     @Path("/echo/{table}")
+    @Authorize(targetType = TargetType.TABLE, paramName = "table", action = Actions.Table.ECHO)

Review Comment:
   Let's remove this one. This is purely for testing purpose



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.pinot.core.auth;
+
+/**
+ * Different action types used in finer grain access control of the rest endpoints
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {

Review Comment:
   Let's order them alphabetically so that same type of action can be grouped together



##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/AuthenticationFilter.java:
##########
@@ -81,10 +85,71 @@ public void filter(ContainerRequestContext requestContext)
 
     HttpRequesterIdentity httpRequestIdentity = HttpRequesterIdentity.fromRequest(request);
 
+    // default authorization handling
     if (!accessControl.hasAccess(httpRequestIdentity)) {
       throw new WebApplicationException("Failed access check for " + httpRequestIdentity.getEndpointUrl(),
           Response.Status.FORBIDDEN);
     }
+
+    handleFinerGrainAuth(endpointMethod, uriInfo, accessControl, httpRequestIdentity);
+  }
+
+  /**
+   * Check for finer grain authorization of APIs.
+   * There are 2 possible cases:
+   * 1. {@link Authorize} annotation is present on the method. In this case, do the finer grain authorization using the
+   *    fields of the annotation. There are 2 possibilities depending on the targetType ({@link TargetType}):
+   *    a. The targetType is {@link TargetType#CLUSTER}. In this case, the paramName field
+   *       ({@link Authorize#paramName()}) is not used, since the target is the Pinot cluster.
+   *    b. The targetType is {@link TargetType#TABLE}. In this case, the paramName field
+   *       ({@link Authorize#paramName()}) is mandatory, and it must be found in either the path parameters or the
+   *       query parameters.
+   * 2. {@link Authorize} annotation is not present on the method. In this use the default authorization.
+   *
+   * @param endpointMethod of the API
+   * @param uriInfo of the API
+   * @param accessControl to check the access
+   * @param httpRequestIdentity of the requester
+   */
+  private void handleFinerGrainAuth(Method endpointMethod, UriInfo uriInfo, AccessControl accessControl,

Review Comment:
   This logic is repeated in 3 places. Can we put it into the Utils?



##########
pinot-controller/src/test/java/org/apache/pinot/controller/api/extraresources/PinotDummyExtraRestletResource.java:
##########
@@ -26,13 +26,17 @@
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
+import org.apache.pinot.core.auth.Actions;
+import org.apache.pinot.core.auth.Authorize;
+import org.apache.pinot.core.auth.TargetType;
 
 
 @Api(tags = "testExtra")
 @Path("/testExtra")
 public class PinotDummyExtraRestletResource {
 
   @GET
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.DUMMY)

Review Comment:
   This is test only API (not visible), do not add it



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.pinot.core.auth;
+
+/**
+ * Different action types used in finer grain access control of the rest endpoints
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String GET_TABLE_LEADERS = "GetTableLeaders";
+    public static final String LIST_TABLES = "ListTables";
+    public static final String RUN_TASK = "RunTask";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+    public static final String GET_APP_CONFIGS = "GetAppConfigs";
+    public static final String GET_ROUTING = "GetRouting";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_LOGGERS = "GetLoggers";
+    public static final String GET_LOGGER = "GetLogger";

Review Comment:
   I don't see the difference between these 2 actions. We don't have logger target type, so we cannot really differentiate these 2 actions. Same for Znode etc. Let's revisit these actions and merge them



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthUtils.java:
##########
@@ -0,0 +1,48 @@
+/**
+ * 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.pinot.core.auth;
+
+import javax.ws.rs.core.MultivaluedMap;
+
+
+/**
+ * Utility methods to share in Broker and Controller request filters.
+ */
+public class RBACAuthUtils {

Review Comment:
   Rename it to `AuthorizationUtils`? I don't think we want to call out RBAC



##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/AuthenticationFilter.java:
##########
@@ -81,10 +85,71 @@ public void filter(ContainerRequestContext requestContext)
 
     HttpRequesterIdentity httpRequestIdentity = HttpRequesterIdentity.fromRequest(request);
 
+    // default authorization handling
     if (!accessControl.hasAccess(httpRequestIdentity)) {
       throw new WebApplicationException("Failed access check for " + httpRequestIdentity.getEndpointUrl(),
           Response.Status.FORBIDDEN);
     }
+
+    handleFinerGrainAuth(endpointMethod, uriInfo, accessControl, httpRequestIdentity);
+  }
+
+  /**
+   * Check for finer grain authorization of APIs.
+   * There are 2 possible cases:
+   * 1. {@link Authorize} annotation is present on the method. In this case, do the finer grain authorization using the
+   *    fields of the annotation. There are 2 possibilities depending on the targetType ({@link TargetType}):
+   *    a. The targetType is {@link TargetType#CLUSTER}. In this case, the paramName field
+   *       ({@link Authorize#paramName()}) is not used, since the target is the Pinot cluster.
+   *    b. The targetType is {@link TargetType#TABLE}. In this case, the paramName field
+   *       ({@link Authorize#paramName()}) is mandatory, and it must be found in either the path parameters or the
+   *       query parameters.
+   * 2. {@link Authorize} annotation is not present on the method. In this use the default authorization.
+   *
+   * @param endpointMethod of the API
+   * @param uriInfo of the API
+   * @param accessControl to check the access
+   * @param httpRequestIdentity of the requester
+   */
+  private void handleFinerGrainAuth(Method endpointMethod, UriInfo uriInfo, AccessControl accessControl,
+      HttpRequesterIdentity httpRequestIdentity) {
+    if (endpointMethod.isAnnotationPresent(Authorize.class)) {
+      final Authorize auth = endpointMethod.getAnnotation(Authorize.class);
+      String targetId = null;
+      // Message to use in the access denied exception
+      String accessDeniedMsg;
+      if (auth.targetType() == TargetType.TABLE) {
+        // paramName is mandatory for table level authorization
+        if (StringUtils.isEmpty(auth.paramName())) {
+          throw new WebApplicationException(
+              "paramName not found for table level authorization in API: " + uriInfo.getRequestUri(),
+              Response.Status.INTERNAL_SERVER_ERROR);
+        }
+
+        // find the paramName in the path or query params
+        targetId = RBACAuthUtils.findParam(auth.paramName(), uriInfo.getPathParameters(), uriInfo.getQueryParameters());

Review Comment:
   Currently sometimes targetId is raw table name (e.g. `myTable`), and sometimes it is table name with type (e.g. `myTable_OFFLINE`). Should we authorize with raw table name? Schema name will never have suffix.



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,187 @@
+/**
+ * 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.pinot.core.auth;
+
+/**
+ * Different action types used in finer grain access control of the rest endpoints
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String GET_TABLE_LEADERS = "GetTableLeaders";
+    public static final String LIST_TABLES = "ListTables";
+    public static final String RUN_TASK = "RunTask";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+    public static final String GET_APP_CONFIGS = "GetAppConfigs";
+    public static final String GET_ROUTING = "GetRouting";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_LOGGERS = "GetLoggers";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String SET_LOGGER = "SetLogger";
+    public static final String GET_LOG_FILES = "GetLogFiles";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String GET_RUNNING_QUERIES = "GetRunningQueries";
+    public static final String LIST_USERS = "ListUsers";
+    public static final String LIST_USER = "ListUser";
+    public static final String ADD_USER = "AddUser";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String GET_CLUSTER_INFO = "GetClusterInfo";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String CHECK_AUTH = "CheckAuth";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_INFO = "GetInfo";
+    public static final String CHECK_HEALTH = "CheckHealth";
+    public static final String LIST_SEGMENTS = "ListSegments";
+    public static final String DUMMY = "Dummy";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String POST_ADMIN_INFO = "PostAdminInfo";
+    public static final String GET_TASK_NAMES = "GetTaskNames";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_INSTANCES = "GetInstances";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String UPDATE_TAGS = "UpdateTags";
+    public static final String UPDATE_RESOURCE = "UpdateResource";
+    public static final String DELETE_QUERY = "DeleteQuery";
+    public static final String GET_SCHEMAS = "GetSchemas";
+    public static final String RESET_SEGMENTS = "ResetSegments";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String LIST_BROKERS = "ListBrokers";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String LIST_TASK_TYPES = "ListTaskTypes";
+    public static final String LIST_TASK_QUEUES = "ListTaskQueues";
+    public static final String GET_TASK_QUEUE_STATE = "GetTaskQueueState";
+    public static final String LIST_TASKS = "ListTasks";
+    public static final String GET_TASK_METADATA = "GetTaskMetadata";
+    public static final String DELETE_TASK_METADATA = "DeleteTaskMetadata";
+    public static final String GET_TASK_COUNTS = "GetTaskCounts";
+    public static final String DEBUG_TASKS = "DebugTasks";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String GET_TASK_STATES = "GetTaskStates";
+    public static final String GET_TASK_STATE = "GetTaskState";
+    public static final String GET_TASK_CONFIG = "GetTaskConfig";
+    public static final String GET_ALL_TASK_STATES = "GetAllTaskStates";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String SCHEDULE_TASK = "ScheduleTask";
+    public static final String SCHEDULE_TASKS = "ScheduleTasks";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String CLEANUP_TASKS = "CleanupTasks";
+    public static final String STOP_TASKS = "StopTasks";
+    public static final String RESUME_TASK = "ResumeTasks";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TASKS = "DeleteTasks";
+    public static final String DELETE_TASK_QUEUE = "DeleteTaskQueue";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String LIST_TENANT = "ListTenant";
+    public static final String LIST_TENANTS = "ListTenants";
+    public static final String GET_TENANT_METADATA = "GetTenantMetadata";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_TABLE_CONFIGS = "GetTableConfigs";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String GET_ZNODES = "GetZnodes";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String LIST_ZNODE = "ListZnode";
+    public static final String LIST_ZNODES = "ListZnodes";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String GET_FIELD_SPEC = "GetFieldSpec";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String ADD_TABLE = "AddTable";
+    public static final String VALIDATE = "Validate";

Review Comment:
   This is confusing. Let's rename the config related actions:
   <GET|UPDATE|VALIDATE|DELETE>TableConfig - TableConfig only
   <GET|UPDATE|VALIDATE|DELETE>Schema - Schema only
   <GET|UPDATE|VALIDATE|DELETE>TableConfigs - TableConfig and Schema (`TableConfigs` class is just a wrapper over table config and schema)



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org