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

[GitHub] [pinot] soumitra-st opened a new pull request, #11016: Rbac impl

soumitra-st opened a new pull request, #11016:
URL: https://github.com/apache/pinot/pull/11016

   Adding support Role Based Access Control in Pinot.
   
   This feature is a work in progress.


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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1248317872


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/AccessControl.java:
##########
@@ -58,4 +58,19 @@ default boolean hasAccess(RequesterIdentity requesterIdentity) {
    * @return {@code true} if authorized, {@code false} otherwise
    */
   boolean hasAccess(RequesterIdentity requesterIdentity, Set<String> tables);
+
+  default boolean hasRBACAccess(RequesterIdentity requesterIdentity, String targetId,

Review Comment:
   Add some javadoc here, especially what does each parameter represent



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthorization.java:
##########
@@ -0,0 +1,33 @@
+/**
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface RBACAuthorization {
+    String targetId() default "";
+    String targetType() default "";
+    String permission() default "";

Review Comment:
   Is this access type?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -381,7 +382,9 @@ private BrokerResponseNative handleRequest(long requestId, String query,
       BrokerRequest brokerRequest = CalciteSqlCompiler.convertToBrokerRequest(pinotQuery);
       BrokerRequest serverBrokerRequest =
           serverPinotQuery == pinotQuery ? brokerRequest : CalciteSqlCompiler.convertToBrokerRequest(serverPinotQuery);
-      boolean hasTableAccess = _accessControlFactory.create().hasAccess(requesterIdentity, serverBrokerRequest);
+      AccessControl accessControl = _accessControlFactory.create();
+      boolean hasTableAccess = accessControl.hasAccess(requesterIdentity, serverBrokerRequest) &&
+              accessControl.hasRBACAccess(requesterIdentity, tableName, "table", "read");

Review Comment:
   `read` seems too general, `query` is more specific



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthorization.java:
##########
@@ -0,0 +1,33 @@
+/**
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface RBACAuthorization {
+    String targetId() default "";

Review Comment:
   Is this parameter name?



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthorization.java:
##########
@@ -0,0 +1,33 @@
+/**
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface RBACAuthorization {
+    String targetId() default "";

Review Comment:
   Please add some javadoc on what each field represents



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/AccessControl.java:
##########
@@ -58,4 +58,19 @@ default boolean hasAccess(RequesterIdentity requesterIdentity) {
    * @return {@code true} if authorized, {@code false} otherwise
    */
   boolean hasAccess(RequesterIdentity requesterIdentity, Set<String> tables);
+
+  default boolean hasRBACAccess(RequesterIdentity requesterIdentity, String targetId,
+                                String targetType, String permission) {

Review Comment:
   (minor) Let's follow [Pinot Style](https://docs.pinot.apache.org/developers/developers-and-contributors/code-setup#setup-ide)



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/AccessControl.java:
##########
@@ -58,4 +58,19 @@ default boolean hasAccess(RequesterIdentity requesterIdentity) {
    * @return {@code true} if authorized, {@code false} otherwise
    */
   boolean hasAccess(RequesterIdentity requesterIdentity, Set<String> tables);
+
+  default boolean hasRBACAccess(RequesterIdentity requesterIdentity, String targetId,
+                                String targetType, String permission) {
+    return true;
+  }
+
+  /**
+   * If an API is neither annotated with RBACAuthorization nor ManualAuthorization,
+   * this method will be called to check the authorization.
+   * If the return is false, then API will be terminated by the filter.
+   * @return true to allow
+   */
+  default boolean defaultAuthorization(RequesterIdentity requesterIdentity) {

Review Comment:
   Seems this is only associated with RBAC access. Can we make its name more specific



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


[GitHub] [pinot] Jackie-Jiang merged pull request #11016: Rbac impl

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #11016:
URL: https://github.com/apache/pinot/pull/11016


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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1273028597


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java:
##########
@@ -129,6 +134,7 @@ public Map<String, Map<ServerInstance, List<String>>> getRoutingTable(
   @GET
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/debug/routingTable/sql")
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.GET_ROUTING)

Review Comment:
   This should be a manual authorize request, where table name can be extracted from the broker request



##########
pinot-integration-tests/src/main/java/org/apache/pinot/broker/api/resources/BrokerEchoWithAutoDiscovery.java:
##########
@@ -28,6 +28,7 @@
 import javax.ws.rs.core.MediaType;
 import org.apache.pinot.core.api.AutoLoadedServiceForTest;
 
+

Review Comment:
   (minor) Revert



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java:
##########
@@ -288,6 +293,7 @@ private void persistInstancePartitionsHelper(InstancePartitions instancePartitio
   @PUT
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables/{tableName}/instancePartitions")
+  @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.ASSIGN_INSTANCE)

Review Comment:
   UPDATE_INSTANCE_PARTITIONS



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java:
##########
@@ -342,6 +348,7 @@ public Map<String, InstancePartitions> setInstancePartitions(
   @DELETE
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables/{tableName}/instancePartitions")
+  @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.DELETE_PARTITION)

Review Comment:
   DELETE_INSTANCE_PARTITIONS



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceRestletResource.java:
##########
@@ -370,6 +379,7 @@ public SuccessResponse updateInstance(
 
   @PUT
   @Path("/instances/{instanceName}/updateTags")
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.UPDATE_TAG)

Review Comment:
   UPDATE_INSTANCE



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java:
##########
@@ -339,6 +363,7 @@ public String validateSchema(String schemaJsonString, @Context HttpHeaders httpH
   @GET
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/schemas/fieldSpec")
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.GET_FIELD_SPEC)

Review Comment:
   This API is kind of special because it does not access any data in the cluster. Is there a way to always allow access?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java:
##########
@@ -149,6 +153,7 @@ public Map<String, InstancePartitions> getInstancePartitions(
   @POST
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables/{tableName}/assignInstances")
+  @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.ASSIGN_INSTANCE)

Review Comment:
   CREATE_INSTANCE_PARTITIONS



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotControllerHealthCheck.java:
##########
@@ -63,6 +66,7 @@ public class PinotControllerHealthCheck {
 
   @GET
   @Path("pinot-controller/admin")
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.GET_INFO)

Review Comment:
   This should be `GET_HEALTH`. Remove `GET_INFO` because it is very ambiguous



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceRestletResource.java:
##########
@@ -407,6 +417,7 @@ public SuccessResponse updateInstanceTags(
 
   @POST
   @Path("/instances/{instanceName}/updateBrokerResource")
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.UPDATE_RESOURCE)

Review Comment:
   UPDATE_BROKER_RESOURCE



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/FineGrainedAuthUtils.java:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 java.lang.reflect.Method;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+
+
+/**
+ * Utility methods to share in Broker and Controller request filters related to fine grain authorization.
+ */
+public class FineGrainedAuthUtils {
+
+  private FineGrainedAuthUtils() {
+  }
+
+  /**
+   * Returns the parameter from the path or query params.
+   * @param paramName to look for
+   * @param pathParams path params
+   * @param queryParams query params
+   * @return the value of the parameter
+   */
+  private static String findParam(String paramName, MultivaluedMap<String, String> pathParams,
+      MultivaluedMap<String, String> queryParams) {
+    String name = pathParams.getFirst(paramName);
+    if (name == null) {
+      name = queryParams.getFirst(paramName);
+    }
+    return name;
+  }
+
+  private static void logAndThrow(Logger logger, String msg, Response.Status status) {

Review Comment:
   Suggest removing this util. I feel it doesn't help with readability



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/FineGrainedAuthUtils.java:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 java.lang.reflect.Method;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import org.apache.commons.lang.StringUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.slf4j.Logger;
+
+
+/**
+ * Utility methods to share in Broker and Controller request filters related to fine grain authorization.
+ */
+public class FineGrainedAuthUtils {
+
+  private FineGrainedAuthUtils() {
+  }
+
+  /**
+   * Returns the parameter from the path or query params.
+   * @param paramName to look for
+   * @param pathParams path params
+   * @param queryParams query params
+   * @return the value of the parameter
+   */
+  private static String findParam(String paramName, MultivaluedMap<String, String> pathParams,
+      MultivaluedMap<String, String> queryParams) {
+    String name = pathParams.getFirst(paramName);
+    if (name == null) {
+      name = queryParams.getFirst(paramName);
+    }
+    return name;
+  }
+
+  private static void logAndThrow(Logger logger, String msg, Response.Status status) {
+    logger.error(msg);

Review Comment:
   We probably don't want to log at error level. For invalid access, currently we don't log anything because it is pure user error



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java:
##########
@@ -82,6 +85,7 @@ public class PinotInstanceAssignmentRestletResource {
   @GET
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables/{tableName}/instancePartitions")
+  @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.GET_PARTITION)

Review Comment:
   GET_INSTANCE_PARTITIONS



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotInstanceAssignmentRestletResource.java:
##########
@@ -395,6 +402,7 @@ private void removeInstancePartitionsHelper(String instancePartitionsName) {
   @POST
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables/{tableName}/replaceInstance")
+  @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.REPLACE_INSTANCE)

Review Comment:
   UPDATE_INSTANCE_PARTITIONS



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,153 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FIELD_SPEC = "GetFieldSpec";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INFO = "GetInfo";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_ROUTING = "GetRouting";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_RESOURCE = "UpdateResource";
+    public static final String UPDATE_TAG = "UpdateTag";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String ASSIGN_INSTANCE = "AssignInstance";
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_PARTITION = "DeletePartition";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DROP_TABLE = "DropTable";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String ENABLE_TABLE = "EnableTable";
+    public static final String FORCE_COMMIT = "ForceCommit";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CONFIG = "GetConfig";
+    public static final String GET_CONSUMING_SEGMENTS = "GetConsumingSegments"; // SK:
+    public static final String GET_CONTROLLER_JOBS = "GetControllerJobs"; // SK:
+    public static final String GET_DEBUG_INFO = "GetDebugInfo";
+    public static final String GET_EXTERNAL_VIEW = "GetExternalView";
+    public static final String GET_IDEAL_STATE = "GetIdealState";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_METADATA = "GetMetadata";
+    public static final String GET_PARTITION = "GetPartition";
+    public static final String GET_PAUSE_STATUS = "GetPauseStatus";
+    public static final String GET_ROUTING_TABLE = "GetRoutingTable";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_LINEAGE = "GetSegmentLineage";
+    public static final String GET_SEGMENT_MAP = "GetSegmentMap";
+    public static final String GET_SERVER_MAP = "GetServerMap";
+    public static final String GET_SIZE = "GetSize";
+    public static final String GET_STATE = "GetState";
+    public static final String GET_STORAGE_TIER = "GetStorageTier";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TIME_BOUNDARY = "GetTimeBoundary";
+    public static final String PAUSE_CONSUMPTION = "PauseConsumption";
+    public static final String QUERY_TABLE = "QueryTable";
+    public static final String REBALANCE_TABLE = "RebalanceTable";
+    public static final String REBUILD_BROKER_RESOURCE = "RebuildBrokerResource";
+    public static final String REFRESH_ROUTING = "RefreshRouting";
+    public static final String RELOAD_SEGMENT = "ReloadSegment";
+    public static final String REPLACE_INSTANCE = "ReplaceInstance";
+    public static final String REPLACE_SEGMENT = "ReplaceSegment";
+    public static final String RESUME_CONSUMPTION = "ResumeConsumption";
+    public static final String UPDATE_CONFIG = "UpdateConfig";
+    public static final String UPDATE_SCHEMA = "UpdateSchema";
+    public static final String UPDATE_TABLE = "UpdateTable";

Review Comment:
   When managing a table, we have 3 scenarios:
   1. TableConfig
   2. Schema
   3. TableConfigs (TableConfig & schema)
   
   We want to name them more explicitly, e.g. `<verb><TableConfig|Schema|TableConfigs>`



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271268475


##########
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:
   No. What I meant is that `Validate` is not accurate. We should use `ValidateTableConfig`, `ValidateSchema`, `ValidateTableConfig`, instead of `Validate` because the target is not included



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on PR #11016:
URL: https://github.com/apache/pinot/pull/11016#issuecomment-1615251276

   Do you have a design doc that can be linked to the PR? Or add more descriptions


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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271254116


##########
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:
   I used schema name in some API to workaround the table-name-wth-type issue. Do you have example APIs where schema or rawtable name should be used? I will review those APIs.



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271254385


##########
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:
   If neither Authorize nor ManualAuthorization is present, then the AUTH call will go to defaultAuthorization and that may not be allowed. If that is the case, no user will be able to use this test API. I suggest to keep the annotation and change the action to Test or Debug, what do you say?



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1275335845


##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_BROKER_RESOURCE = "UpdateBrokerResource";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_INSTANCE_PARTITIONS = "CreateInstancePartitions";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_INSTANCE_PARTITIONS = "DeleteInstancePartitions";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    public static final String ENABLE_TABLE = "EnableTable";
+    public static final String FORCE_COMMIT = "ForceCommit";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CONFIG = "GetConfig";
+    public static final String GET_CONSUMING_SEGMENTS = "GetConsumingSegments";
+    public static final String GET_CONTROLLER_JOBS = "GetControllerJobs";
+    public static final String GET_DEBUG_INFO = "GetDebugInfo";
+    public static final String GET_EXTERNAL_VIEW = "GetExternalView";
+    public static final String GET_IDEAL_STATE = "GetIdealState";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_INSTANCE_PARTITIONS = "GetInstancePartitions";
+    public static final String GET_METADATA = "GetMetadata";
+    public static final String GET_PAUSE_STATUS = "GetPauseStatus";
+    public static final String GET_ROUTING_TABLE = "GetRoutingTable";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_LINEAGE = "GetSegmentLineage";
+    public static final String GET_SEGMENT_MAP = "GetSegmentMap";
+    public static final String GET_SERVER_MAP = "GetServerMap";
+    public static final String GET_SIZE = "GetSize";
+    public static final String GET_STATE = "GetState";
+    public static final String GET_STORAGE_TIER = "GetStorageTier";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TIME_BOUNDARY = "GetTimeBoundary";
+    public static final String PAUSE_CONSUMPTION = "PauseConsumption";
+    public static final String QUERY = "Query";
+    public static final String REBALANCE_TABLE = "RebalanceTable";
+    public static final String REBUILD_BROKER_RESOURCE = "RebuildBrokerResource";
+    public static final String REFRESH_ROUTING = "RefreshRouting";
+    public static final String RELOAD_SEGMENT = "ReloadSegment";
+    public static final String REPLACE_SEGMENT = "ReplaceSegment";
+    public static final String RESUME_CONSUMPTION = "ResumeConsumption";
+    public static final String UPDATE_INSTANCE_PARTITIONS = "UpdateInstancePartitions";
+    public static final String UPDATE_SCHEMA = "UpdateSchema";
+    public static final String UPDATE_TABLE_CONFIG = "UpdateTableConfig";
+    public static final String UPDATE_TABLE_CONFIGS = "UpdateTableConfigs";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+    public static final String VALIDATE_CONFIG = "ValidateConfig";

Review Comment:
   For consistency, suggest rename this to `VALIDATE_TABLE_CONFIGS`



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_BROKER_RESOURCE = "UpdateBrokerResource";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_INSTANCE_PARTITIONS = "CreateInstancePartitions";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_INSTANCE_PARTITIONS = "DeleteInstancePartitions";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    public static final String ENABLE_TABLE = "EnableTable";
+    public static final String FORCE_COMMIT = "ForceCommit";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CONFIG = "GetConfig";

Review Comment:
   For consistency, suggest rename this to `GET_TABLE_CONFIGS`



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java:
##########
@@ -153,10 +153,15 @@ public Map<ServerInstance, List<String>> getRoutingTableForQuery(
       @Context HttpHeaders httpHeaders) {
     BrokerRequest brokerRequest = CalciteSqlCompiler.compileToBrokerRequest(query);
 
-    if (!_accessControlFactory.create()
-        .hasAccess(httpHeaders, TargetType.TABLE, brokerRequest.getQuerySource().getTableName(),
-            Actions.Table.GET_ROUTING)) {
-      throw new WebApplicationException("Permission denied", Response.Status.FORBIDDEN);
+    // TODO: Handle nested queries
+    if (brokerRequest.isSetPinotQuery() && brokerRequest.getPinotQuery().dataSource.isSetTableName()) {

Review Comment:
   This can still cause NPE because `dataSource` might not be set in `PinotQuery`
   ```suggestion
       if (brokerRequest.isSetQuerySource() && brokerRequest.getQuerySource().isSetTableName()) {
   ```



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1275368362


##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_BROKER_RESOURCE = "UpdateBrokerResource";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_INSTANCE_PARTITIONS = "CreateInstancePartitions";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_INSTANCE_PARTITIONS = "DeleteInstancePartitions";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    public static final String ENABLE_TABLE = "EnableTable";
+    public static final String FORCE_COMMIT = "ForceCommit";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CONFIG = "GetConfig";
+    public static final String GET_CONSUMING_SEGMENTS = "GetConsumingSegments";
+    public static final String GET_CONTROLLER_JOBS = "GetControllerJobs";
+    public static final String GET_DEBUG_INFO = "GetDebugInfo";
+    public static final String GET_EXTERNAL_VIEW = "GetExternalView";
+    public static final String GET_IDEAL_STATE = "GetIdealState";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_INSTANCE_PARTITIONS = "GetInstancePartitions";
+    public static final String GET_METADATA = "GetMetadata";
+    public static final String GET_PAUSE_STATUS = "GetPauseStatus";
+    public static final String GET_ROUTING_TABLE = "GetRoutingTable";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_LINEAGE = "GetSegmentLineage";
+    public static final String GET_SEGMENT_MAP = "GetSegmentMap";
+    public static final String GET_SERVER_MAP = "GetServerMap";
+    public static final String GET_SIZE = "GetSize";
+    public static final String GET_STATE = "GetState";
+    public static final String GET_STORAGE_TIER = "GetStorageTier";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TIME_BOUNDARY = "GetTimeBoundary";
+    public static final String PAUSE_CONSUMPTION = "PauseConsumption";
+    public static final String QUERY = "Query";
+    public static final String REBALANCE_TABLE = "RebalanceTable";
+    public static final String REBUILD_BROKER_RESOURCE = "RebuildBrokerResource";
+    public static final String REFRESH_ROUTING = "RefreshRouting";
+    public static final String RELOAD_SEGMENT = "ReloadSegment";
+    public static final String REPLACE_SEGMENT = "ReplaceSegment";
+    public static final String RESUME_CONSUMPTION = "ResumeConsumption";
+    public static final String UPDATE_INSTANCE_PARTITIONS = "UpdateInstancePartitions";
+    public static final String UPDATE_SCHEMA = "UpdateSchema";
+    public static final String UPDATE_TABLE_CONFIG = "UpdateTableConfig";
+    public static final String UPDATE_TABLE_CONFIGS = "UpdateTableConfigs";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+    public static final String VALIDATE_CONFIG = "ValidateConfig";

Review Comment:
   We have GET_TABLE_CONFIG, GET_TABLE_CONFIGS and UPDATE_TABLE_CONFIG and UPDATE_TABLE_CONFIGS, shall we have one version only, i.e.
   GET_TABLE_CONFIG
   UPDATE_TABLE_CONFIG
   
   and
   
   remove GET_TABLE_CONFIGS and UPDATE_TABLE_CONFIGS ?



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


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

Posted by "zhtaoxiang (via GitHub)" <gi...@apache.org>.
zhtaoxiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1253713691


##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthUtils.java:
##########
@@ -0,0 +1,38 @@
+/**
+ * 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;
+
+public class RBACAuthUtils {
+    public static String getTargetId(String targetIdTemplate,

Review Comment:
   can we please add java doc here?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/AuthenticationFilter.java:
##########
@@ -81,10 +83,32 @@ 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);
     }
+
+    handleRBACAuthorization(endpointMethod, uriInfo, accessControl, httpRequestIdentity);
+  }
+
+  private void handleRBACAuthorization(Method endpointMethod, UriInfo uriInfo, AccessControl accessControl,
+                                       HttpRequesterIdentity httpRequestIdentity) {
+    if (endpointMethod.isAnnotationPresent(RBACAuthorization.class)) {
+      RBACAuthorization rbacAuthorization = endpointMethod.getAnnotation(RBACAuthorization.class);
+      String targetId = RBACAuthUtils.getTargetId(rbacAuthorization.targetId(), uriInfo.getPathParameters(),

Review Comment:
   I feel that we should use different names for the variable `targetId` and `rbacAuthorization.targetId()`, it is confusing to me now.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseBrokerRequestHandler.java:
##########
@@ -381,7 +382,9 @@ private BrokerResponseNative handleRequest(long requestId, String query,
       BrokerRequest brokerRequest = CalciteSqlCompiler.convertToBrokerRequest(pinotQuery);
       BrokerRequest serverBrokerRequest =
           serverPinotQuery == pinotQuery ? brokerRequest : CalciteSqlCompiler.convertToBrokerRequest(serverPinotQuery);
-      boolean hasTableAccess = _accessControlFactory.create().hasAccess(requesterIdentity, serverBrokerRequest);
+      AccessControl accessControl = _accessControlFactory.create();
+      boolean hasTableAccess = accessControl.hasAccess(requesterIdentity, serverBrokerRequest) &&
+              accessControl.hasRBACAccess(requesterIdentity, tableName, "table", "read");

Review Comment:
   We may need to define some static constant variables to represent the types and permissions.



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1274197978


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotControllerPeriodicTaskRestletResource.java:
##########
@@ -57,9 +61,13 @@ public class PinotControllerPeriodicTaskRestletResource {
   @Inject
   PeriodicTaskScheduler _periodicTaskScheduler;
 
+  @Inject
+  AccessControlFactory _accessControlFactory;

Review Comment:
   (minor) Is this used?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java:
##########
@@ -129,16 +141,25 @@ public Map<String, Map<ServerInstance, List<String>>> getRoutingTable(
   @GET
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/debug/routingTable/sql")
+  @ManualAuthorization
   @ApiOperation(value = "Get the routing table for a query")
   @ApiResponses(value = {
       @ApiResponse(code = 200, message = "Routing table"),
       @ApiResponse(code = 404, message = "Routing not found"),
       @ApiResponse(code = 500, message = "Internal server error")
   })
   public Map<ServerInstance, List<String>> getRoutingTableForQuery(
-      @ApiParam(value = "SQL query (table name should have type suffix)") @QueryParam("query") String query) {
-    RoutingTable routingTable = _routingManager.getRoutingTable(CalciteSqlCompiler.compileToBrokerRequest(query),
-        getRequestId());
+      @ApiParam(value = "SQL query (table name should have type suffix)") @QueryParam("query") String query,
+      @Context HttpHeaders httpHeaders) {
+    BrokerRequest brokerRequest = CalciteSqlCompiler.compileToBrokerRequest(query);
+
+    if (!_accessControlFactory.create()
+        .hasAccess(httpHeaders, TargetType.TABLE, brokerRequest.getQuerySource().getTableName(),
+            Actions.Table.GET_ROUTING)) {

Review Comment:
   This should also be `GET_ROUTING_TABLE`



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,150 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_BROKER_RESOURCE = "UpdateBrokerResource";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_INSTANCE_PARTITIONS = "CreateInstancePartitions";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_INSTANCE_PARTITIONS = "DeleteInstancePartitions";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DROP_TABLE = "DropTable";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String ENABLE_TABLE = "EnableTable";
+    public static final String FORCE_COMMIT = "ForceCommit";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CONFIG = "GetConfig";
+    public static final String GET_CONSUMING_SEGMENTS = "GetConsumingSegments";
+    public static final String GET_CONTROLLER_JOBS = "GetControllerJobs";
+    public static final String GET_DEBUG_INFO = "GetDebugInfo";
+    public static final String GET_EXTERNAL_VIEW = "GetExternalView";
+    public static final String GET_IDEAL_STATE = "GetIdealState";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_INSTANCE_PARTITIONS = "GetInstancePartitions";
+    public static final String GET_METADATA = "GetMetadata";
+    public static final String GET_PAUSE_STATUS = "GetPauseStatus";
+    public static final String GET_ROUTING = "GetRouting";
+    public static final String GET_ROUTING_TABLE = "GetRoutingTable";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_LINEAGE = "GetSegmentLineage";
+    public static final String GET_SEGMENT_MAP = "GetSegmentMap";
+    public static final String GET_SERVER_MAP = "GetServerMap";
+    public static final String GET_SIZE = "GetSize";
+    public static final String GET_STATE = "GetState";
+    public static final String GET_STORAGE_TIER = "GetStorageTier";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TIME_BOUNDARY = "GetTimeBoundary";
+    public static final String PAUSE_CONSUMPTION = "PauseConsumption";
+    public static final String QUERY_TABLE = "QueryTable";

Review Comment:
   Should we just call it `Query`?



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,150 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_BROKER_RESOURCE = "UpdateBrokerResource";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_INSTANCE_PARTITIONS = "CreateInstancePartitions";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_INSTANCE_PARTITIONS = "DeleteInstancePartitions";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DROP_TABLE = "DropTable";

Review Comment:
   This is the same as `DeleteTable`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -386,6 +402,11 @@ public String alterTableStateOrListTableConfig(
       String endpointUrl = request.getRequestURL().toString();
       AccessControlUtils.validatePermission(tableName, AccessType.UPDATE, httpHeaders, endpointUrl,
           _accessControlFactory.create());
+      // Convert the action to EnableTable, DisableTable and DropTable corresponding to StateType enum.
+      if (!_accessControlFactory.create().hasAccess(httpHeaders, TargetType.TABLE, tableName,

Review Comment:
   Suggest doing a switch on `stateType` and use the constant from `Actions` for readability and easier code navigation.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -562,6 +585,12 @@ public ObjectNode checkTableConfig(String tableConfigStr,
     String endpointUrl = request.getRequestURL().toString();
     AccessControlUtils.validatePermission(tableName, AccessType.READ, httpHeaders, endpointUrl,
         _accessControlFactory.create());
+    // Using schema name, since table name contains type (OFFLINE) suffix

Review Comment:
   Use table name



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -597,6 +626,10 @@ public String validateTableAndSchema(TableAndSchemaConfig tableSchemaConfig,
     String endpointUrl = request.getRequestURL().toString();
     AccessControlUtils.validatePermission(schemaName, AccessType.READ, httpHeaders, endpointUrl,
         _accessControlFactory.create());
+    if (!_accessControlFactory.create()
+        .hasAccess(httpHeaders, TargetType.TABLE, schemaName, Actions.Table.VALIDATE_TABLE)) {

Review Comment:
   Use table name. `schemaName` can be `null`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -200,6 +203,12 @@ public ConfigSuccessResponse addTable(String tableConfigStr,
       String endpointUrl = request.getRequestURL().toString();
       AccessControlUtils.validatePermission(tableName, AccessType.CREATE, httpHeaders, endpointUrl,
           _accessControlFactory.create());
+      // Using schema name, since table name contains type (OFFLINE) suffix

Review Comment:
   We want to use table name here. Schema name might be `null`



##########
pinot-core/src/main/java/org/apache/pinot/core/auth/Actions.java:
##########
@@ -0,0 +1,150 @@
+/**
+ * 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
+ * Action names are in <verb><noun> format, e.g. GetSchema, ListTables, etc.
+ */
+public class Actions {
+  // Action names for cluster
+  public static class Cluster {
+    public static final String CANCEL_QUERY = "CancelQuery";
+    public static final String CLEANUP_TASK = "CleanupTask";
+    public static final String COMMIT_SEGMENT = "CommitSegment";
+    public static final String CREATE_INSTANCE = "CreateInstance";
+    public static final String CREATE_TASK = "CreateTask";
+    public static final String CREATE_TENANT = "CreateTenant";
+    public static final String CREATE_USER = "CreateUser";
+    public static final String DEBUG_TASK = "DebugTask";
+    public static final String DELETE_CLUSTER_CONFIG = "DeleteClusterConfig";
+    public static final String DELETE_INSTANCE = "DeleteInstance";
+    public static final String DELETE_TASK = "DeleteTask";
+    public static final String DELETE_TENANT = "DeleteTenant";
+    public static final String DELETE_USER = "DeleteUser";
+    public static final String DELETE_ZNODE = "DeleteZnode";
+    public static final String ESTIMATE_UPSERT_MEMORY = "EstimateUpsertMemory";
+    public static final String EXECUTE_TASK = "ExecuteTask";
+    public static final String GET_ADMIN_INFO = "GetAdminInfo";
+    public static final String GET_APP_CONFIG = "GetAppConfig";
+    public static final String GET_AUTH = "GetAuth";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CLUSTER_CONFIG = "GetClusterConfig";
+    public static final String GET_FORCE_COMMIT_STATUS = "GetForceCommitStatus";
+    public static final String GET_HEALTH = "GetHealth";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_LOGGER = "GetLogger";
+    public static final String GET_LOG_FILE = "GetLogFile";
+    public static final String GET_REBALANCE_STATUS = "GetRebalanceStatus";
+    public static final String GET_RUNNING_QUERY = "GetRunningQuery";
+    public static final String GET_SCHEDULER_INFO = "GetSchedulerInfo";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_RELOAD_STATUS = "GetSegmentReloadStatus";
+    public static final String GET_SERVER_ROUTING_STATS = "GetServerRoutingStats";
+    public static final String GET_TABLE = "GetTable";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TASK = "GetTask";
+    public static final String GET_TENANT = "GetTenant";
+    public static final String GET_USER = "GetUser";
+    public static final String GET_VERSION = "GetVersion";
+    public static final String GET_ZNODE = "GetZnode";
+    public static final String INGEST_FILE = "IngestFile";
+    public static final String RECOMMEND_CONFIG = "RecommendConfig";
+    public static final String RESET_SEGMENT = "ResetSegment";
+    public static final String RESUME_TASK = "ResumeTask";
+    public static final String STOP_TASK = "StopTask";
+    public static final String UPDATE_BROKER_RESOURCE = "UpdateBrokerResource";
+    public static final String UPDATE_CLUSTER_CONFIG = "UpdateClusterConfig";
+    public static final String UPDATE_INSTANCE = "UpdateInstance";
+    public static final String UPDATE_LOGGER = "UpdateLogger";
+    public static final String UPDATE_QPS = "UpdateQPS";
+    public static final String UPDATE_TASK_QUEUE = "UpdateTaskQueue";
+    public static final String UPDATE_TENANT = "UpdateTenant";
+    public static final String UPDATE_TENANT_METADATA = "UpdateTenantMetadata";
+    public static final String UPDATE_TIME_INTERVAL = "UpdateTimeInterval";
+    public static final String UPDATE_USER = "UpdateUser";
+    public static final String UPDATE_ZNODE = "UpdateZnode";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+  }
+
+  // Action names for table
+  public static class Table {
+    public static final String BUILD_ROUTING = "BuildRouting";
+    public static final String CREATE_INSTANCE_PARTITIONS = "CreateInstancePartitions";
+    public static final String CREATE_SCHEMA = "CreateSchema";
+    public static final String CREATE_TABLE = "CreateTable";
+    public static final String DELETE_INSTANCE_PARTITIONS = "DeleteInstancePartitions";
+    public static final String DELETE_ROUTING = "DeleteRouting";
+    public static final String DELETE_SCHEMA = "DeleteSchema";
+    public static final String DELETE_SEGMENT = "DeleteSegment";
+    public static final String DELETE_TABLE = "DeleteTable";
+    public static final String DELETE_TIME_BOUNDARY = "DeleteTimeBoundary";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DISABLE_TABLE = "DisableTable";
+    public static final String DOWNLOAD_SEGMENT = "DownloadSegment";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String DROP_TABLE = "DropTable";
+    // Used in /tables/{tableName} API with state as one of the parameters
+    public static final String ENABLE_TABLE = "EnableTable";
+    public static final String FORCE_COMMIT = "ForceCommit";
+    public static final String GET_BROKER = "GetBroker";
+    public static final String GET_CONFIG = "GetConfig";
+    public static final String GET_CONSUMING_SEGMENTS = "GetConsumingSegments";
+    public static final String GET_CONTROLLER_JOBS = "GetControllerJobs";
+    public static final String GET_DEBUG_INFO = "GetDebugInfo";
+    public static final String GET_EXTERNAL_VIEW = "GetExternalView";
+    public static final String GET_IDEAL_STATE = "GetIdealState";
+    public static final String GET_INSTANCE = "GetInstance";
+    public static final String GET_INSTANCE_PARTITIONS = "GetInstancePartitions";
+    public static final String GET_METADATA = "GetMetadata";
+    public static final String GET_PAUSE_STATUS = "GetPauseStatus";
+    public static final String GET_ROUTING = "GetRouting";
+    public static final String GET_ROUTING_TABLE = "GetRoutingTable";
+    public static final String GET_SCHEMA = "GetSchema";
+    public static final String GET_SEGMENT = "GetSegment";
+    public static final String GET_SEGMENT_LINEAGE = "GetSegmentLineage";
+    public static final String GET_SEGMENT_MAP = "GetSegmentMap";
+    public static final String GET_SERVER_MAP = "GetServerMap";
+    public static final String GET_SIZE = "GetSize";
+    public static final String GET_STATE = "GetState";
+    public static final String GET_STORAGE_TIER = "GetStorageTier";
+    public static final String GET_TABLE_CONFIG = "GetTableConfig";
+    public static final String GET_TABLE_LEADER = "GetTableLeader";
+    public static final String GET_TIME_BOUNDARY = "GetTimeBoundary";
+    public static final String PAUSE_CONSUMPTION = "PauseConsumption";
+    public static final String QUERY_TABLE = "QueryTable";
+    public static final String REBALANCE_TABLE = "RebalanceTable";
+    public static final String REBUILD_BROKER_RESOURCE = "RebuildBrokerResource";
+    public static final String REFRESH_ROUTING = "RefreshRouting";
+    public static final String RELOAD_SEGMENT = "ReloadSegment";
+    public static final String REPLACE_SEGMENT = "ReplaceSegment";
+    public static final String RESUME_CONSUMPTION = "ResumeConsumption";
+    public static final String UPDATE_INSTANCE_PARTITIONS = "UpdateInstancePartitions";
+    public static final String UPDATE_SCHEMA = "UpdateSchema";
+    public static final String UPDATE_TABLE_CONFIG = "UpdateTableConfig";
+    public static final String UPDATE_TABLE_CONFIGS = "UpdateTableConfigs";
+    public static final String UPLOAD_SEGMENT = "UploadSegment";
+    public static final String VALIDATE_CONFIG = "ValidateConfig";
+    public static final String VALIDATE_SCHEMA = "ValidateSchema";
+    public static final String VALIDATE_TABLE = "ValidateTable";

Review Comment:
   This is the same as `ValidateConfig`



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotBrokerDebug.java:
##########
@@ -129,16 +141,25 @@ public Map<String, Map<ServerInstance, List<String>>> getRoutingTable(
   @GET
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/debug/routingTable/sql")
+  @ManualAuthorization
   @ApiOperation(value = "Get the routing table for a query")
   @ApiResponses(value = {
       @ApiResponse(code = 200, message = "Routing table"),
       @ApiResponse(code = 404, message = "Routing not found"),
       @ApiResponse(code = 500, message = "Internal server error")
   })
   public Map<ServerInstance, List<String>> getRoutingTableForQuery(
-      @ApiParam(value = "SQL query (table name should have type suffix)") @QueryParam("query") String query) {
-    RoutingTable routingTable = _routingManager.getRoutingTable(CalciteSqlCompiler.compileToBrokerRequest(query),
-        getRequestId());
+      @ApiParam(value = "SQL query (table name should have type suffix)") @QueryParam("query") String query,
+      @Context HttpHeaders httpHeaders) {
+    BrokerRequest brokerRequest = CalciteSqlCompiler.compileToBrokerRequest(query);
+
+    if (!_accessControlFactory.create()
+        .hasAccess(httpHeaders, TargetType.TABLE, brokerRequest.getQuerySource().getTableName(),

Review Comment:
   This can cause NPE (both `QuerySource` and `TableName` might be `null`). We can add a null check for now, and reject access when table name is not available. Please add a TODO to handle nested query in the future



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271269196


##########
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:
   What I mean is that should we always use raw table name to authorize?
   The `tableName` param usually take both raw table name and table name with type, and I don't see a scenario where a user is authorized to only the OFFLINE table but not the REALTIME table. If we want to support that, we need to define the access for raw table name and table name with type separately.



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271268534


##########
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:
   Same as the `echo`, this resource file is in the test package, so not included in the production code



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271268324


##########
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:
   This resource is in the test package for testing purpose only, and is not included in the production code, thus not visible to the user



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271545265


##########
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:
   Got it. Shall we leave that to the implementation of [AccessControl.hasAccess](https://github.com/apache/pinot/pull/11016/files#diff-5d02b4239775fe072ec911349f56c32e7ca2d6cd77226d6239b1511096d57e2cR72) to remove the trailing _OFFLINE/_REALTIME and validate the authorization on the raw table name if that is what someone wants to do? As such from the framework perspective Pinot should send the actual table name used in the API call. What do you think?



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1266042072


##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthorization.java:
##########
@@ -0,0 +1,33 @@
+/**
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface RBACAuthorization {
+    String targetId() default "";

Review Comment:
   Changed the annotation to Authorize.



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271579047


##########
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:
   I'd suggest just handling it here with `TableNameBuilder.extractRawTableName()` when the target type is table. Currently there is no way for us to differentiate whether the param is raw table name or table name with type 



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271255084


##########
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:
   There are two TableConfig related API, one at cluster level to get all table configs, and other at table level, are you suggesting to change them to TableConfig?
   
   Are you suggesting to remove GetSchema, DeleteSchema, UpdateSchema, etc. and use Schema action for all of them?



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271255206


##########
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:
   If Authorize annotation is removed, no user may be able call it if default auth returns false. I would advise to change the action to Test or Debug instead.



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1271726836


##########
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:
   Added an interface for fine-grained access control to avoid code duplication in controller and broker request filters.



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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1266042275


##########
pinot-core/src/main/java/org/apache/pinot/core/auth/RBACAuthorization.java:
##########
@@ -0,0 +1,33 @@
+/**
+ * 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface RBACAuthorization {
+    String targetId() default "";
+    String targetType() default "";
+    String permission() default "";

Review Comment:
   Yes, changed to action in Authorize annotation.



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


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

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
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


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

Posted by "soumitra-st (via GitHub)" <gi...@apache.org>.
soumitra-st commented on code in PR #11016:
URL: https://github.com/apache/pinot/pull/11016#discussion_r1273623477


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java:
##########
@@ -339,6 +363,7 @@ public String validateSchema(String schemaJsonString, @Context HttpHeaders httpH
   @GET
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/schemas/fieldSpec")
+  @Authorize(targetType = TargetType.CLUSTER, action = Actions.Cluster.GET_FIELD_SPEC)

Review Comment:
   Yes, annotated with ManualAuthorization to skip from request filter, and no validation in the implementation.



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


[GitHub] [pinot] codecov-commenter commented on pull request #11016: Rbac impl

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #11016:
URL: https://github.com/apache/pinot/pull/11016#issuecomment-1614927288

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#11016](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (ed6dced) into [master](https://app.codecov.io/gh/apache/pinot/commit/cc5a9bec7d4341533613b87e62707a9ceabc0cd0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (cc5a9be) will **increase** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #11016     +/-   ##
   =========================================
     Coverage    0.11%    0.11%             
   =========================================
     Files        2191     2139     -52     
     Lines      118007   115549   -2458     
     Branches    17868    17576    -292     
   =========================================
     Hits          137      137             
   + Misses     117850   115392   -2458     
     Partials       20       20             
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1temurin11 | `?` | |
   | integration1temurin17 | `?` | |
   | integration1temurin20 | `?` | |
   | integration2temurin11 | `?` | |
   | integration2temurin17 | `?` | |
   | integration2temurin20 | `?` | |
   | unittests1temurin11 | `?` | |
   | unittests1temurin20 | `?` | |
   | unittests2temurin11 | `?` | |
   | unittests2temurin17 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   | unittests2temurin20 | `0.11% <0.00%> (-0.01%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...ava/org/apache/pinot/broker/api/AccessControl.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL0FjY2Vzc0NvbnRyb2wuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...ache/pinot/broker/broker/AuthenticationFilter.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL0F1dGhlbnRpY2F0aW9uRmlsdGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...roker/requesthandler/BaseBrokerRequestHandler.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvQmFzZUJyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...che/pinot/controller/api/access/AccessControl.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvYWNjZXNzL0FjY2Vzc0NvbnRyb2wuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...ot/controller/api/access/AuthenticationFilter.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvYWNjZXNzL0F1dGhlbnRpY2F0aW9uRmlsdGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...not/controller/api/resources/PinotTableSchema.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90VGFibGVTY2hlbWEuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [...java/org/apache/pinot/core/auth/RBACAuthUtils.java](https://app.codecov.io/gh/apache/pinot/pull/11016?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9hdXRoL1JCQUNBdXRoVXRpbHMuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   
   ... and [58 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/11016/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


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