diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 096a1439a4fd..442563d305be 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -49,6 +49,7 @@ import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.DirectStatement.ResultSet; @@ -146,7 +147,7 @@ private QueryResponse runNativeQuery(QueryRequest request, AuthenticationResult final String currThreadName = Thread.currentThread().getName(); try { queryLifecycle.initialize(query); - Access authorizationResult = queryLifecycle.authorize(authResult); + AuthorizationResult authorizationResult = queryLifecycle.authorize(authResult); if (!authorizationResult.isAllowed()) { throw new ForbiddenException(Access.DEFAULT_ERROR_MESSAGE); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityResourceFilter.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityResourceFilter.java index bcb4ec053457..1c7d374da308 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityResourceFilter.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityResourceFilter.java @@ -23,7 +23,7 @@ import com.sun.jersey.spi.container.ContainerRequest; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.http.security.AbstractResourceFilter; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; @@ -54,7 +54,7 @@ public ContainerRequest filter(ContainerRequest request) getAction(request) ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, getAuthorizerMapper() @@ -64,7 +64,7 @@ public ContainerRequest filter(ContainerRequest request) throw new WebApplicationException( Response.status(Response.Status.FORBIDDEN) .type(MediaType.TEXT_PLAIN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .entity(StringUtils.format("Access-Check-Result: %s", authResult.getFailureMessage())) .build() ); } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 2590f57c8b38..0eb9ebf58867 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -34,8 +34,8 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -56,10 +56,10 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; - import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; /** @@ -108,17 +108,17 @@ public CatalogResource( * * * @param schemaName The name of the Druid schema, which must be writable - * and the user must have at least read access. - * @param tableName The name of the table definition to modify. The user must - * have write access to the table. - * @param spec The new table definition. - * @param version the expected version of an existing table. The version must - * match. If not (or if the table does not exist), returns an error. - * @param overwrite if {@code true}, then overwrites any existing table. - * If {@code false}, then the operation fails if the table already exists. - * Ignored if a version is specified. - * @param req the HTTP request used for authorization. - */ + * and the user must have at least read access. + * @param tableName The name of the table definition to modify. The user must + * have write access to the table. + * @param spec The new table definition. + * @param version the expected version of an existing table. The version must + * match. If not (or if the table does not exist), returns an error. + * @param overwrite if {@code true}, then overwrites any existing table. + * If {@code false}, then the operation fails if the table already exists. + * Ignored if a version is specified. + * @param req the HTTP request used for authorization. + */ @POST @Path("/schemas/{schema}/tables/{name}") @Consumes(MediaType.APPLICATION_JSON) @@ -181,9 +181,9 @@ public Response postTable( * the definition is created before the datasource itself.) * * @param schemaName The Druid schema. The user must have read access. - * @param tableName The name of the table within the schema. The user must have - * read access. - * @param req the HTTP request used for authorization. + * @param tableName The name of the table within the schema. The user must have + * read access. + * @param req the HTTP request used for authorization. * @return the definition for the table, if any. */ @GET @@ -211,8 +211,8 @@ public Response getTable( * for the given schema and table. * * @param schemaName The name of the schema that holds the table. - * @param tableName The name of the table definition to delete. The user must have - * write access. + * @param tableName The name of the table definition to delete. The user must have + * write access. */ @DELETE @Path("/schemas/{schema}/tables/{name}") @@ -247,9 +247,9 @@ public Response deleteTable( * the table spec changed between the time it was retrieve and the edit operation * is submitted. * - * @param schemaName The name of the schema that holds the table. - * @param tableName The name of the table definition to delete. The user must have - * write access. + * @param schemaName The name of the schema that holds the table. + * @param tableName The name of the table definition to delete. The user must have + * write access. * @param editRequest The operation to perform. See the classes for details. */ @POST @@ -281,7 +281,7 @@ public Response editTable( * Retrieves the list of all Druid schema names. * * @param format the format of the response. See the code for the - * available formats + * available formats */ @GET @Path("/schemas") @@ -318,9 +318,9 @@ public Response getSchemas( * the read-only schemas, there will be no table definitions. * * @param schemaName The name of the Druid schema to query. The user must - * have read access. - * @param format the format of the response. See the code for the - * available formats + * have read access. + * @param format the format of the response. See the code for the + * available formats */ @GET @Path("/schemas/{schema}/tables") @@ -360,7 +360,7 @@ public Response getSchemaTables( * table definitions known to the catalog. Used to prime a cache on first access. * After that, the Coordinator will push updates to Brokers. Returns the full * list of table details. - * + *

* It is expected that the number of table definitions will be of small or moderate * size, so no provision is made to handle very large lists. */ @@ -467,9 +467,9 @@ private Response listAllTableMetadata(final HttpServletRequest req) List> tables = new ArrayList<>(); for (SchemaSpec schema : catalog.schemaRegistry().schemas()) { tables.addAll(catalog.tables().tablesInSchema(schema.name()) - .stream() - .map(table -> Pair.of(schema, table)) - .collect(Collectors.toList())); + .stream() + .map(table -> Pair.of(schema, table)) + .collect(Collectors.toList())); } Iterable> filtered = AuthorizationUtils.filterAuthorizedResources( @@ -483,9 +483,9 @@ private Response listAllTableMetadata(final HttpServletRequest req) ); List metadata = Lists.newArrayList(filtered) - .stream() - .map(pair -> pair.rhs) - .collect(Collectors.toList()); + .stream() + .map(pair -> pair.rhs) + .collect(Collectors.toList()); return Response.ok().entity(metadata).build(); } @@ -499,9 +499,9 @@ private Response tableNamesInSchema( req, tables, name -> - Collections.singletonList( - resourceAction(schema, name, Action.READ)), - authorizerMapper + Collections.singletonList( + resourceAction(schema, name, Action.READ)), + authorizerMapper ); return Response.ok().entity(Lists.newArrayList(filtered)).build(); } @@ -581,13 +581,13 @@ private void authorizeTable( private void authorize(String resource, String key, Action action, HttpServletRequest request) { - final Access authResult = authorizeAccess(resource, key, action, request); + final AuthorizationResult authResult = authorizeAccess(resource, key, action, request); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } } - private Access authorizeAccess(String resource, String key, Action action, HttpServletRequest request) + private AuthorizationResult authorizeAccess(String resource, String key, Action action, HttpServletRequest request) { return AuthorizationUtils.authorizeResourceAction( request, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java index 65d770a29c55..97a1f32f66a9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java @@ -36,9 +36,9 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.ResponseContextConfig; import org.apache.druid.server.initialization.ServerConfig; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; @@ -144,7 +144,7 @@ public GetQueriesResponse doGetRunningQueries( ) { final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); - final Access stateReadAccess = AuthorizationUtils.authorizeAllResourceActions( + final AuthorizationResult stateReadAccess = AuthorizationUtils.authorizeAllResourceActions( authenticationResult, Collections.singletonList(new ResourceAction(Resource.STATE_RESOURCE, Action.READ)), authorizerMapper @@ -245,7 +245,7 @@ public Response cancelQuery( return Response.status(Response.Status.ACCEPTED).build(); } - final Access access = authorizeCancellation(req, cancelables); + final AuthorizationResult access = authorizeCancellation(req, cancelables); if (access.isAllowed()) { sqlLifecycleManager.removeAll(sqlQueryId, cancelables); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java index 8820b4ead5a0..7d9779b723f5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/MSQResourceUtils.java @@ -19,7 +19,7 @@ package org.apache.druid.msq.rpc; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -27,6 +27,7 @@ import javax.servlet.http.HttpServletRequest; import java.util.List; +import java.util.Objects; /** * Utility methods for MSQ resources such as {@link ControllerResource}. @@ -41,10 +42,14 @@ public static void authorizeAdminRequest( { final List resourceActions = permissionMapper.getAdminPermissions(); - Access access = AuthorizationUtils.authorizeAllResourceActions(request, resourceActions, authorizerMapper); + AuthorizationResult access = AuthorizationUtils.authorizeAllResourceActions( + request, + resourceActions, + authorizerMapper + ); if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); + throw new ForbiddenException(Objects.requireNonNull(access.getFailureMessage())); } } @@ -57,10 +62,14 @@ public static void authorizeQueryRequest( { final List resourceActions = permissionMapper.getQueryPermissions(queryId); - Access access = AuthorizationUtils.authorizeAllResourceActions(request, resourceActions, authorizerMapper); + AuthorizationResult access = AuthorizationUtils.authorizeAllResourceActions( + request, + resourceActions, + authorizerMapper + ); if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); + throw new ForbiddenException(Objects.requireNonNull(access.getFailureMessage())); } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index c92bfa955fb6..33527e6eb903 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -74,9 +74,9 @@ import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.QueryResponse; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -484,7 +484,13 @@ private Response buildTaskResponse(Sequence sequence, AuthenticationRe } String taskId = String.valueOf(firstRow[0]); - Optional statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ, false); + Optional statementResult = getStatementStatus( + taskId, + authenticationResult, + true, + Action.READ, + false + ); if (statementResult.isPresent()) { return Response.status(Response.Status.OK).entity(statementResult.get()).build(); @@ -585,7 +591,11 @@ private Optional getStatementStatus( } // since we need the controller payload for auth checks. - MSQControllerTask msqControllerTask = getMSQControllerTaskAndCheckPermission(queryId, authenticationResult, forAction); + MSQControllerTask msqControllerTask = getMSQControllerTaskAndCheckPermission( + queryId, + authenticationResult, + forAction + ); SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); MSQTaskReportPayload taskReportPayload = null; @@ -640,9 +650,9 @@ private Optional getStatementStatus( * necessary permissions. A user has the necessary permissions if one of the following criteria is satisfied: * 1. The user is the one who submitted the query * 2. The user belongs to a role containing the READ or WRITE permissions over the STATE resource. For endpoints like GET, - * the user should have READ permission for the STATE resource, while for endpoints like DELETE, the user should - * have WRITE permission for the STATE resource. (Note: POST API does not need to check the state permissions since - * the currentUser always equal to the queryUser) + * the user should have READ permission for the STATE resource, while for endpoints like DELETE, the user should + * have WRITE permission for the STATE resource. (Note: POST API does not need to check the state permissions since + * the currentUser always equal to the queryUser) */ private MSQControllerTask getMSQControllerTaskAndCheckPermission( String queryId, @@ -665,7 +675,7 @@ private MSQControllerTask getMSQControllerTaskAndCheckPermission( return msqControllerTask; } - Access access = AuthorizationUtils.authorizeAllResourceActions( + AuthorizationResult access = AuthorizationUtils.authorizeAllResourceActions( authenticationResult, Collections.singletonList(new ResourceAction(Resource.STATE_RESOURCE, forAction)), authorizerMapper @@ -990,7 +1000,11 @@ private T contactOverlord(final ListenableFuture future, String queryId) private static DruidException queryNotFoundException(String queryId) { - return NotFound.exception("Query [%s] was not found. The query details are no longer present or might not be of the type [%s]. Verify that the id is correct.", queryId, MSQControllerTask.TYPE); + return NotFound.exception( + "Query [%s] was not found. The query details are no longer present or might not be of the type [%s]. Verify that the id is correct.", + queryId, + MSQControllerTask.TYPE + ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java index 79a3e8993a8c..70b1b50e2daf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java @@ -27,8 +27,8 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.ParseExceptionReport; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -43,6 +43,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; public class IndexTaskUtils { @@ -67,7 +68,7 @@ public static List getReportListFromSavedParseExceptions( * * @return authorization result */ - public static Access datasourceAuthorizationCheck( + public static AuthorizationResult datasourceAuthorizationCheck( final HttpServletRequest req, Action action, String datasource, @@ -79,9 +80,9 @@ public static Access datasourceAuthorizationCheck( action ); - Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + AuthorizationResult access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); + throw new ForbiddenException(Objects.requireNonNull(access.getFailureMessage())); } return access; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 84bf260b6f99..b432132e0f27 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -66,9 +66,9 @@ import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.http.security.DatasourceResourceFilter; import org.apache.druid.server.http.security.StateResourceFilter; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -101,6 +101,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; /** @@ -177,14 +178,14 @@ public Response taskPost( .build(); } - Access authResult = AuthorizationUtils.authorizeAllResourceActions( + AuthorizationResult authResult = AuthorizationUtils.authorizeAllResourceActions( req, resourceActions, authorizerMapper ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.getMessage()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return asLeaderWith( @@ -609,7 +610,7 @@ public Response getTasks( new Resource(dataSource, ResourceType.DATASOURCE), Action.READ ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( req, resourceAction, authorizerMapper @@ -618,7 +619,10 @@ public Response getTasks( throw new WebApplicationException( Response.status(Response.Status.FORBIDDEN) .type(MediaType.TEXT_PLAIN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .entity(StringUtils.format( + "Access-Check-Result: %s", + Objects.requireNonNull(authResult.getFailureMessage()) + )) .build() ); } @@ -654,7 +658,7 @@ public Response killPendingSegments( { final Interval deleteInterval = Intervals.of(deleteIntervalString); // check auth for dataSource - final Access authResult = AuthorizationUtils.authorizeAllResourceActions( + final AuthorizationResult authResult = AuthorizationUtils.authorizeAllResourceActions( request, ImmutableList.of( new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.READ), @@ -664,7 +668,7 @@ public Response killPendingSegments( ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.getMessage()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } if (overlord.isLeader()) { @@ -678,7 +682,12 @@ public Response killPendingSegments( .build(); } catch (Exception e) { - log.warn(e, "Failed to delete pending segments for datasource[%s] and interval[%s].", dataSource, deleteInterval); + log.warn( + e, + "Failed to delete pending segments for datasource[%s] and interval[%s].", + dataSource, + deleteInterval + ); return Response.status(Status.INTERNAL_SERVER_ERROR) .entity(ImmutableMap.of("error", e.getMessage())) .build(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/SupervisorResourceFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/SupervisorResourceFilter.java index 7e9aaa927407..b0647623c509 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/SupervisorResourceFilter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/SupervisorResourceFilter.java @@ -30,8 +30,8 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.http.security.AbstractResourceFilter; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -41,6 +41,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.PathSegment; import javax.ws.rs.core.Response; +import java.util.Objects; public class SupervisorResourceFilter extends AbstractResourceFilter { @@ -97,14 +98,14 @@ public boolean apply(PathSegment input) AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR : AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR; - Access authResult = AuthorizationUtils.authorizeAllResourceActions( + AuthorizationResult authResult = AuthorizationUtils.authorizeAllResourceActions( getReq(), Iterables.transform(spec.getDataSources(), resourceActionFunction), getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return request; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java index a9f66ce30e72..bd63c1197814 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/security/TaskResourceFilter.java @@ -29,7 +29,7 @@ import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.http.security.AbstractResourceFilter; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -40,6 +40,7 @@ import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.util.Objects; /** * Use this ResourceFilter when the datasource information is present after "task" segment in the request Path @@ -92,14 +93,14 @@ public ContainerRequest filter(ContainerRequest request) getAction(request) ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return request; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResource.java index 75618ddae42a..2b4fa1565328 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResource.java @@ -23,9 +23,9 @@ import com.google.inject.Inject; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -40,6 +40,7 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import java.util.HashSet; +import java.util.Objects; import java.util.Set; @Path("/druid/indexer/v1/sampler") @@ -72,14 +73,14 @@ public SamplerResponse post(final SamplerSpec sampler, @Context final HttpServle resourceActions.addAll(sampler.getInputSourceResources()); } - Access authResult = AuthorizationUtils.authorizeAllResourceActions( + AuthorizationResult authResult = AuthorizationUtils.authorizeAllResourceActions( req, resourceActions, authorizerMapper ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.getMessage()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return sampler.sample(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index 130f617d59d1..dc1bc41bbebd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -41,9 +41,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.segment.incremental.ParseExceptionReport; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -142,14 +142,14 @@ public Response specPost(final SupervisorSpec spec, @Context final HttpServletRe .build(); } - Access authResult = AuthorizationUtils.authorizeAllResourceActions( + AuthorizationResult authResult = AuthorizationUtils.authorizeAllResourceActions( req, resourceActions, authorizerMapper ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } manager.createOrUpdateAndStartSupervisor(spec); @@ -410,13 +410,16 @@ public Response shutdown(@PathParam("id") final String id) @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(SupervisorResourceFilter.class) - public Response handoffTaskGroups(@PathParam("id") final String id, @Nonnull final HandoffTaskGroupsRequest handoffTaskGroupsRequest) + public Response handoffTaskGroups( + @PathParam("id") final String id, + @Nonnull final HandoffTaskGroupsRequest handoffTaskGroupsRequest + ) { List taskGroupIds = handoffTaskGroupsRequest.getTaskGroupIds(); if (CollectionUtils.isNullOrEmpty(taskGroupIds)) { return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", "List of task groups to handoff can't be empty")) - .build(); + .entity(ImmutableMap.of("error", "List of task groups to handoff can't be empty")) + .build(); } return asLeaderWithSupervisorManager( @@ -426,14 +429,20 @@ public Response handoffTaskGroups(@PathParam("id") final String id, @Nonnull fin return Response.ok().build(); } else { return Response.status(Response.Status.NOT_FOUND) - .entity(ImmutableMap.of("error", StringUtils.format("Supervisor was not found [%s]", id))) - .build(); + .entity(ImmutableMap.of("error", StringUtils.format("Supervisor was not found [%s]", id))) + .build(); } } catch (NotImplementedException e) { return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", StringUtils.format("Supervisor [%s] does not support early handoff", id))) - .build(); + .entity(ImmutableMap.of( + "error", + StringUtils.format( + "Supervisor [%s] does not support early handoff", + id + ) + )) + .build(); } } ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index a2db12d005e9..7ee3dcb6f9da 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -89,8 +89,8 @@ import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CollectionUtils; @@ -144,7 +144,8 @@ * @param Sequence Number Type */ @SuppressWarnings("CheckReturnValue") -public abstract class SeekableStreamIndexTaskRunner implements ChatHandler +public abstract class SeekableStreamIndexTaskRunner + implements ChatHandler { private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; @@ -278,12 +279,11 @@ public SeekableStreamIndexTaskRunner( rejectionPeriodUpdaterExec = Execs.scheduledSingleThreaded("RejectionPeriodUpdater-Exec--%d"); if (ioConfig.getRefreshRejectionPeriodsInMinutes() != null) { - rejectionPeriodUpdaterExec - .scheduleWithFixedDelay( - this::refreshMinMaxMessageTime, - ioConfig.getRefreshRejectionPeriodsInMinutes(), - ioConfig.getRefreshRejectionPeriodsInMinutes(), - TimeUnit.MINUTES); + rejectionPeriodUpdaterExec.scheduleWithFixedDelay(this::refreshMinMaxMessageTime, + ioConfig.getRefreshRejectionPeriodsInMinutes(), + ioConfig.getRefreshRejectionPeriodsInMinutes(), + TimeUnit.MINUTES + ); } resetNextCheckpointTime(); } @@ -759,10 +759,18 @@ public void onFailure(Throwable t) if (System.currentTimeMillis() > nextCheckpointTime) { sequenceToCheckpoint = getLastSequenceMetadata(); - log.info("Next checkpoint time, updating sequenceToCheckpoint, SequenceToCheckpoint: [%s]", sequenceToCheckpoint); + log.info( + "Next checkpoint time, updating sequenceToCheckpoint, SequenceToCheckpoint: [%s]", + sequenceToCheckpoint + ); } if (pushTriggeringAddResult != null) { - log.info("Hit the row limit updating sequenceToCheckpoint, SequenceToCheckpoint: [%s], rowInSegment: [%s], TotalRows: [%s]", sequenceToCheckpoint, pushTriggeringAddResult.getNumRowsInSegment(), pushTriggeringAddResult.getTotalNumRowsInAppenderator()); + log.info( + "Hit the row limit updating sequenceToCheckpoint, SequenceToCheckpoint: [%s], rowInSegment: [%s], TotalRows: [%s]", + sequenceToCheckpoint, + pushTriggeringAddResult.getNumRowsInSegment(), + pushTriggeringAddResult.getTotalNumRowsInAppenderator() + ); } if (sequenceToCheckpoint != null && stillReading) { @@ -1128,14 +1136,14 @@ private synchronized void persistSequences() throws IOException /** * Return a map of reports for the task. - * + *

* A successfull task should always have a null errorMsg. Segments availability is inherently confirmed * if the task was succesful. - * + *

* A falied task should always have a non-null errorMsg. Segment availability is never confirmed if the task * was not successful. * - * @param errorMsg Nullable error message for the task. null if task succeeded. + * @param errorMsg Nullable error message for the task. null if task succeeded. * @param handoffWaitMs Milliseconds waited for segments to be handed off. * @return Map of reports for the task. */ @@ -1446,7 +1454,7 @@ protected void sendResetRequestAndWait( * * @return authorization result */ - private Access authorizationCheck(final HttpServletRequest req, Action action) + private AuthorizationResult authorizationCheck(final HttpServletRequest req, Action action) { return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper); } @@ -2023,9 +2031,7 @@ private boolean verifyRecordInRange( * * @param toolbox task toolbox * @param checkpointsString the json-serialized checkpoint string - * * @return checkpoint - * * @throws IOException jsonProcessingException */ @Nullable @@ -2039,7 +2045,6 @@ protected abstract TreeMap> ge * This is what would become the start offsets of the next reader, if we stopped reading now. * * @param sequenceNumber the sequence number that has already been processed - * * @return next sequence number to be stored */ protected abstract SequenceOffsetType getNextStartOffset(SequenceOffsetType sequenceNumber); @@ -2049,7 +2054,6 @@ protected abstract TreeMap> ge * * @param mapper json objectMapper * @param object metadata - * * @return SeekableStreamEndSequenceNumbers */ protected abstract SeekableStreamEndSequenceNumbers deserializePartitionsFromMetadata( @@ -2063,9 +2067,7 @@ protected abstract SeekableStreamEndSequenceNumbers createDataSourceMetadata( @@ -2089,7 +2090,6 @@ protected abstract SeekableStreamDataSourceMetadata createSequenceNumber(SequenceOffsetType sequenceNumber); @@ -2117,7 +2117,11 @@ private void refreshMinMaxMessageTime() minMessageTime = minMessageTime.plusMinutes(ioConfig.getRefreshRejectionPeriodsInMinutes().intValue()); maxMessageTime = maxMessageTime.plusMinutes(ioConfig.getRefreshRejectionPeriodsInMinutes().intValue()); - log.info(StringUtils.format("Updated min and max messsage times to %s and %s respectively.", minMessageTime, maxMessageTime)); + log.info(StringUtils.format( + "Updated min and max messsage times to %s and %s respectively.", + minMessageTime, + maxMessageTime + )); } public boolean withinMinMaxRecordTime(final InputRow row) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 1fd7af69e123..f07c6c13ab88 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -114,13 +114,13 @@ public Authorizer getAuthorizer(String name) } else { if (resource.getType().equals(ResourceType.DATASOURCE)) { if (resource.getName().equals("datasource2")) { - return new Access(false, "not authorized."); + return Access.deny("not authorized."); } else { return Access.OK; } } else if (resource.getType().equals(ResourceType.EXTERNAL)) { if (resource.getName().equals("test")) { - return new Access(false, "not authorized."); + return Access.deny("not authorized."); } else { return Access.OK; } diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 360c339627f9..ca990d4a09f6 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,14 +21,18 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.SegmentReference; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +import java.util.stream.Collectors; /** * Represents a source... of data... for a query. Analogous to the "FROM" clause in SQL. @@ -43,7 +47,8 @@ @JsonSubTypes.Type(value = InlineDataSource.class, name = "inline"), @JsonSubTypes.Type(value = GlobalTableDataSource.class, name = "globalTable"), @JsonSubTypes.Type(value = UnnestDataSource.class, name = "unnest"), - @JsonSubTypes.Type(value = FilteredDataSource.class, name = "filter") + @JsonSubTypes.Type(value = FilteredDataSource.class, name = "filter"), + @JsonSubTypes.Type(value = RestrictedDataSource.class, name = "restrict") }) public interface DataSource { @@ -88,11 +93,11 @@ public interface DataSource /** * Returns true if this datasource can be the base datasource of query processing. - * + *

* Base datasources drive query processing. If the base datasource is {@link TableDataSource}, for example, queries * are processed in parallel on data servers. If the base datasource is {@link InlineDataSource}, queries are * processed on the Broker. See {@link DataSourceAnalysis#getBaseDataSource()} for further discussion. - * + *

* Datasources that are *not* concrete must be pre-processed in some way before they can be processed by the main * query stack. For example, {@link QueryDataSource} must be executed first and substituted with its results. * @@ -118,6 +123,22 @@ public interface DataSource */ DataSource withUpdatedDataSource(DataSource newSource); + /** + * Returns an updated datasource based on the policy restrictions on tables. If this datasource contains no table, no + * changes should occur. + * + * @param rowFilters a mapping of table names to row filters, every table in the datasource tree must have an entry + * @return the updated datasource, with restrictions applied in the datasource tree + */ + default DataSource mapWithRestriction(Map> rowFilters) + { + List children = this.getChildren() + .stream() + .map(child -> child.mapWithRestriction(rowFilters)) + .collect(Collectors.toList()); + return this.withChildren(children); + } + /** * Compute a cache key prefix for a data source. This includes the data sources that participate in the RHS of a * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 220f18a94855..65ec304c8b8e 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -529,7 +529,10 @@ private static Triple> flattenJoi // Will need an instanceof check here // A future work should look into if the flattenJoin // can be refactored to omit these instanceof checks - while (current instanceof JoinDataSource || current instanceof UnnestDataSource || current instanceof FilteredDataSource) { + while (current instanceof JoinDataSource + || current instanceof UnnestDataSource + || current instanceof FilteredDataSource + || current instanceof RestrictedDataSource) { if (current instanceof JoinDataSource) { final JoinDataSource joinDataSource = (JoinDataSource) current; current = joinDataSource.getLeft(); @@ -545,6 +548,9 @@ private static Triple> flattenJoi } else if (current instanceof UnnestDataSource) { final UnnestDataSource unnestDataSource = (UnnestDataSource) current; current = unnestDataSource.getBase(); + } else if (current instanceof RestrictedDataSource) { + final RestrictedDataSource restrictedDataSource = (RestrictedDataSource) current; + current = restrictedDataSource.getBase(); } else { final FilteredDataSource filteredDataSource = (FilteredDataSource) current; current = filteredDataSource.getBase(); diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 3ed1dcbe0ead..e1700402c442 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -54,6 +54,7 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -242,6 +243,11 @@ default String getMostSpecificId() Query withDataSource(DataSource dataSource); + default Query withPolicyRestrictions(Map> restrictions) + { + return this.withDataSource(this.getDataSource().mapWithRestriction(restrictions)); + } + default Query optimizeForSegment(PerSegmentQueryOptimizationContext optimizationContext) { return this; diff --git a/processing/src/main/java/org/apache/druid/query/RestrictedDataSource.java b/processing/src/main/java/org/apache/druid/query/RestrictedDataSource.java new file mode 100644 index 000000000000..277410e091f3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/RestrictedDataSource.java @@ -0,0 +1,227 @@ +/* + * 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.druid.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.TrueDimFilter; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.RestrictedSegment; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.utils.JvmUtils; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; + +/** + * Reperesents a TableDataSource with row-level policy restriction. + *

+ * A RestrictedDataSource means the base TableDataSource has policy imposed. A table without any policy should never be + * transformed to a RestrictedDataSource. Druid internal system and admin users would have a null rowFilter, while + * external users would have a rowFilter based on the applied policy. + */ +public class RestrictedDataSource implements DataSource +{ + private final TableDataSource base; + @Nullable + private final DimFilter rowFilter; + + @JsonProperty("base") + public TableDataSource getBase() + { + return base; + } + + /** + * Returns true if the row-level filter imposes no restrictions. + */ + public boolean allowAll() + { + return Objects.isNull(rowFilter) || rowFilter.equals(TrueDimFilter.instance()); + } + + @Nullable + @JsonProperty("filter") + public DimFilter getFilter() + { + return rowFilter; + } + + RestrictedDataSource(TableDataSource base, @Nullable DimFilter rowFilter) + { + this.base = base; + this.rowFilter = rowFilter; + } + + @JsonCreator + public static RestrictedDataSource create( + @JsonProperty("base") DataSource base, + @Nullable @JsonProperty("filter") DimFilter rowFilter + ) + { + if (!(base instanceof TableDataSource)) { + throw new IAE("Expected a TableDataSource, got [%s]", base.getClass()); + } + return new RestrictedDataSource((TableDataSource) base, rowFilter); + } + + @Override + public Set getTableNames() + { + return base.getTableNames(); + } + + @Override + public List getChildren() + { + return ImmutableList.of(base); + } + + @Override + public DataSource withChildren(List children) + { + if (children.size() != 1) { + throw new IAE("Expected [1] child, got [%d]", children.size()); + } + + return RestrictedDataSource.create(children.get(0), rowFilter); + } + + @Override + public boolean isCacheable(boolean isBroker) + { + return false; + } + + @Override + public boolean isGlobal() + { + return base.isGlobal(); + } + + @Override + public boolean isConcrete() + { + return base.isConcrete(); + } + + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTimeAccumulator + ) + { + return JvmUtils.safeAccumulateThreadCpuTime( + cpuTimeAccumulator, + () -> base.createSegmentMapFunction( + query, + cpuTimeAccumulator + ).andThen((segment) -> (new RestrictedSegment(segment, rowFilter))) + ); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return RestrictedDataSource.create(newSource, rowFilter); + } + + @Override + public DataSource mapWithRestriction(Map> rowFilters) + { + if (!rowFilters.containsKey(this.base.getName())) { + throw DruidException.defensive("Missing row filter for table [%s]", this.base.getName()); + } + + Optional newFilter = rowFilters.get(this.base.getName()); + if (newFilter.isEmpty()) { + throw DruidException.defensive( + "No restriction found on table [%s], but had %s before.", + this.base.getName(), + this.rowFilter + ); + } + if (newFilter.get().equals(TrueDimFilter.instance())) { + // The internal druid_system always has a TrueDimFilter, whic can be applied in conjunction with an external user's filter. + return this; + } else if (newFilter.get().equals(rowFilter)) { + // This likely occurs when we perform an authentication check for the same user more than once, which is not ideal. + return this; + } else { + throw new ISE("Incompatible restrictions on [%s]: %s and %s", this.base.getName(), rowFilter, newFilter.get()); + } + } + + @Override + public String toString() + { + try { + return "RestrictedDataSource{" + + "base=" + base + + ", filter='" + rowFilter + '}'; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public byte[] getCacheKey() + { + return new byte[0]; + } + + @Override + public DataSourceAnalysis getAnalysis() + { + final DataSource current = this.getBase(); + return current.getAnalysis(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RestrictedDataSource that = (RestrictedDataSource) o; + return Objects.equals(base, that.base) && Objects.equals(rowFilter, that.rowFilter); + } + + @Override + public int hashCode() + { + return Objects.hash(base, rowFilter); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index fe9cf46e37b9..a69792885a13 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -23,13 +23,18 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.SegmentReference; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; @@ -112,6 +117,20 @@ public DataSource withUpdatedDataSource(DataSource newSource) return newSource; } + @Override + public DataSource mapWithRestriction(Map> rowFilters) + { + if (!rowFilters.containsKey(this.name)) { + throw DruidException.defensive("Need to check row-level policy for all tables, missing [%s]", this.name); + } + Optional filter = rowFilters.get(this.name); + if (filter.isEmpty()) { + // Skip adding restriction on table if there's no policy restriction found. + return this; + } + return RestrictedDataSource.create(this, filter.get().equals(TrueDimFilter.instance()) ? null : filter.get()); + } + @Override public byte[] getCacheKey() { diff --git a/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java index 8314e71308f2..8b2b13e4aece 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java @@ -40,7 +40,6 @@ public static NotDimFilter of(DimFilter field) return new NotDimFilter(field); } - private final DimFilter field; @JsonCreator diff --git a/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java index f2d434bab8a5..600d9c805c37 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -30,6 +30,7 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; +import org.apache.druid.query.RestrictedDataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; import org.apache.druid.query.filter.DimFilter; @@ -74,7 +75,7 @@ public static AnalysisType fromString(String name) @Override public byte[] getCacheKey() { - return new byte[] {(byte) this.ordinal()}; + return new byte[]{(byte) this.ordinal()}; } } @@ -118,7 +119,9 @@ public SegmentMetadataQuery( if (lenientAggregatorMerge != null && aggregatorMergeStrategy != null) { throw InvalidInput.exception("Both lenientAggregatorMerge [%s] and aggregatorMergeStrategy [%s] parameters cannot be set." + " Consider using aggregatorMergeStrategy since lenientAggregatorMerge is deprecated.", - lenientAggregatorMerge, aggregatorMergeStrategy); + lenientAggregatorMerge, + aggregatorMergeStrategy + ); } if (lenientAggregatorMerge != null) { this.aggregatorMergeStrategy = lenientAggregatorMerge @@ -220,6 +223,11 @@ public Query withQuerySegmentSpec(QuerySegmentSpec spec) @Override public Query withDataSource(DataSource dataSource) { + if (dataSource instanceof RestrictedDataSource && ((RestrictedDataSource) dataSource).allowAll()) { + return Druids.SegmentMetadataQueryBuilder.copy(this) + .dataSource(((RestrictedDataSource) dataSource).getBase()) + .build(); + } return Druids.SegmentMetadataQueryBuilder.copy(this).dataSource(dataSource).build(); } @@ -249,14 +257,14 @@ public List getIntervals() public String toString() { return "SegmentMetadataQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", toInclude=" + toInclude + - ", merge=" + merge + - ", usingDefaultInterval=" + usingDefaultInterval + - ", analysisTypes=" + analysisTypes + - ", aggregatorMergeStrategy=" + aggregatorMergeStrategy + - '}'; + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", toInclude=" + toInclude + + ", merge=" + merge + + ", usingDefaultInterval=" + usingDefaultInterval + + ", analysisTypes=" + analysisTypes + + ", aggregatorMergeStrategy=" + aggregatorMergeStrategy + + '}'; } @Override @@ -273,10 +281,10 @@ public boolean equals(Object o) } SegmentMetadataQuery that = (SegmentMetadataQuery) o; return merge == that.merge && - usingDefaultInterval == that.usingDefaultInterval && - Objects.equals(toInclude, that.toInclude) && - Objects.equals(analysisTypes, that.analysisTypes) && - Objects.equals(aggregatorMergeStrategy, that.aggregatorMergeStrategy); + usingDefaultInterval == that.usingDefaultInterval && + Objects.equals(toInclude, that.toInclude) && + Objects.equals(analysisTypes, that.analysisTypes) && + Objects.equals(aggregatorMergeStrategy, that.aggregatorMergeStrategy); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java new file mode 100644 index 000000000000..f29fce0780d5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/RestrictedCursorFactory.java @@ -0,0 +1,74 @@ +/* + * 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.druid.segment; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.filter.AndFilter; + +import javax.annotation.Nullable; + +public class RestrictedCursorFactory implements CursorFactory +{ + private final CursorFactory delegate; + @Nullable + private final DimFilter filter; + + public RestrictedCursorFactory( + CursorFactory delegate, + @Nullable DimFilter filter + ) + { + this.delegate = delegate; + this.filter = filter; + } + + @Override + public CursorHolder makeCursorHolder(CursorBuildSpec spec) + { + if (filter == null) { + return delegate.makeCursorHolder(spec); + } + + final CursorBuildSpec.CursorBuildSpecBuilder buildSpecBuilder = CursorBuildSpec.builder(spec); + final Filter newFilter = spec.getFilter() == null + ? filter.toFilter() + : new AndFilter(ImmutableList.of(spec.getFilter(), filter.toFilter())); + buildSpecBuilder.setFilter(newFilter); + + return delegate.makeCursorHolder(buildSpecBuilder.build()); + } + + @Override + public RowSignature getRowSignature() + { + return delegate.getRowSignature(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(column); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java b/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java new file mode 100644 index 000000000000..2004a1eaeb30 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/RestrictedSegment.java @@ -0,0 +1,45 @@ +/* + * 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.druid.segment; + +import org.apache.druid.query.filter.DimFilter; + +import javax.annotation.Nullable; + +public class RestrictedSegment extends WrappedSegmentReference +{ + @Nullable + private final DimFilter filter; + + public RestrictedSegment( + SegmentReference delegate, + @Nullable DimFilter filter + ) + { + super(delegate); + this.filter = filter; + } + + @Override + public CursorFactory asCursorFactory() + { + return new RestrictedCursorFactory(delegate.asCursorFactory(), filter); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java index e7850953a609..1f5e955f5754 100644 --- a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java @@ -20,16 +20,21 @@ package org.apache.druid.query; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import java.util.Optional; public class DataSourceTest { @@ -61,6 +66,16 @@ public void testTableDataSource() throws IOException Assert.assertEquals(new TableDataSource("somedatasource"), dataSource); } + @Test + public void testRestrictedDataSource() throws IOException + { + DataSource dataSource = JSON_MAPPER.readValue( + "{\"type\":\"restrict\",\"base\":{\"type\":\"table\",\"name\":\"somedatasource\"},\"filter\":null}", + DataSource.class + ); + Assert.assertEquals(RestrictedDataSource.create(TableDataSource.create("somedatasource"), null), dataSource); + } + @Test public void testQueryDataSource() throws IOException { @@ -99,4 +114,65 @@ public void testUnionDataSource() throws Exception final DataSource serde = JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(dataSource), DataSource.class); Assert.assertEquals(dataSource, serde); } + + @Test + public void testMapWithRestriction() throws Exception + { + TableDataSource table1 = TableDataSource.create("table1"); + TableDataSource table2 = TableDataSource.create("table2"); + TableDataSource table3 = TableDataSource.create("table3"); + UnionDataSource unionDataSource = new UnionDataSource(Lists.newArrayList(table1, table2, table3)); + ImmutableMap> restrictions = ImmutableMap.of( + "table1", + Optional.of(TrueDimFilter.instance()), + "table2", + Optional.empty(), + "table3", + Optional.of(new NullFilter( + "some-column", + null + )) + ); + + Assert.assertEquals( + unionDataSource.mapWithRestriction(restrictions), + new UnionDataSource(Lists.newArrayList( + RestrictedDataSource.create(table1, null), + table2, + RestrictedDataSource.create(table3, new NullFilter("some-column", null)) + )) + ); + } + + @Test + public void testMapWithRestrictionThrowsWhenMissingRestriction() throws Exception + { + TableDataSource table1 = TableDataSource.create("table1"); + TableDataSource table2 = TableDataSource.create("table2"); + UnionDataSource unionDataSource = new UnionDataSource(Lists.newArrayList(table1, table2)); + ImmutableMap> restrictions = ImmutableMap.of( + "table1", + Optional.of(TrueDimFilter.instance()) + ); + + Exception e = Assert.assertThrows(RuntimeException.class, () -> unionDataSource.mapWithRestriction(restrictions)); + Assert.assertEquals(e.getMessage(), "Need to check row-level policy for all tables, missing [table2]"); + } + + @Test + public void testMapWithRestrictionThrowsWithIncompatibleRestriction() throws Exception + { + RestrictedDataSource restrictedDataSource = RestrictedDataSource.create(TableDataSource.create("table1"), null); + ImmutableMap> restrictions = ImmutableMap.of( + "table1", + Optional.of(new NullFilter("some-column", null)) + ); + + Assert.assertThrows(RuntimeException.class, () -> restrictedDataSource.mapWithRestriction(restrictions)); + Assert.assertThrows(RuntimeException.class, () -> restrictedDataSource.mapWithRestriction(ImmutableMap.of())); + Assert.assertThrows( + RuntimeException.class, + () -> restrictedDataSource.mapWithRestriction(ImmutableMap.of("table1", Optional.empty())) + ); + } } diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index b821bc49c4e7..382a85ba5a94 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -481,6 +481,26 @@ public void testGetAnalysisWithFilteredDS() Assert.assertEquals("table1", analysis.getBaseDataSource().getTableNames().iterator().next()); } + @Test + public void testGetAnalysisWithRestrictedDS() + { + JoinDataSource dataSource = JoinDataSource.create( + RestrictedDataSource.create( + new TableDataSource("table1"), + TrueDimFilter.instance() + ), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + null + ); + DataSourceAnalysis analysis = dataSource.getAnalysis(); + Assert.assertEquals("table1", analysis.getBaseDataSource().getTableNames().iterator().next()); + } + @Test public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() { diff --git a/processing/src/test/java/org/apache/druid/query/RestrictedDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/RestrictedDataSourceTest.java new file mode 100644 index 000000000000..6239729d20cd --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/RestrictedDataSourceTest.java @@ -0,0 +1,154 @@ +/* + * 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.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; + +public class RestrictedDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final TableDataSource fooDataSource = new TableDataSource("foo"); + private final TableDataSource barDataSource = new TableDataSource("bar"); + private final RestrictedDataSource restrictedFooDataSource = RestrictedDataSource.create(fooDataSource, null); + private final RestrictedDataSource restrictedBarDataSource = RestrictedDataSource.create(barDataSource, null); + + @Test + public void test_getTableNames() + { + Assert.assertEquals(Collections.singleton("foo"), restrictedFooDataSource.getTableNames()); + Assert.assertEquals(Collections.singleton("bar"), restrictedBarDataSource.getTableNames()); + } + + @Test + public void test_getChildren() + { + Assert.assertEquals(Collections.singletonList(fooDataSource), restrictedFooDataSource.getChildren()); + Assert.assertEquals(Collections.singletonList(barDataSource), restrictedBarDataSource.getChildren()); + } + + @Test + public void test_isCacheable() + { + Assert.assertFalse(restrictedFooDataSource.isCacheable(true)); + } + + @Test + public void test_isGlobal() + { + Assert.assertFalse(restrictedFooDataSource.isGlobal()); + } + + @Test + public void test_isConcrete() + { + Assert.assertTrue(restrictedFooDataSource.isConcrete()); + } + + @Test + public void test_withChildren() + { + IllegalArgumentException exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> restrictedFooDataSource.withChildren(Collections.emptyList()) + ); + Assert.assertEquals(exception.getMessage(), "Expected [1] child, got [0]"); + + IllegalArgumentException exception2 = Assert.assertThrows( + IllegalArgumentException.class, + () -> restrictedFooDataSource.withChildren(ImmutableList.of(fooDataSource, barDataSource)) + ); + Assert.assertEquals(exception2.getMessage(), "Expected [1] child, got [2]"); + + RestrictedDataSource newRestrictedDataSource = (RestrictedDataSource) restrictedFooDataSource.withChildren( + ImmutableList.of(barDataSource)); + Assert.assertEquals(newRestrictedDataSource.getBase(), barDataSource); + } + + @Test + public void test_withUpdatedDataSource() + { + RestrictedDataSource newRestrictedDataSource = (RestrictedDataSource) restrictedFooDataSource.withUpdatedDataSource( + new TableDataSource("bar")); + Assert.assertEquals(newRestrictedDataSource.getBase(), barDataSource); + } + + @Test + public void test_withAnalysis() + { + Assert.assertEquals(restrictedFooDataSource.getAnalysis(), fooDataSource.getAnalysis()); + Assert.assertEquals(restrictedBarDataSource.getAnalysis(), barDataSource.getAnalysis()); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(RestrictedDataSource.class).usingGetClass().withNonnullFields("base").verify(); + } + + @Test + public void test_serde_roundTrip() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final RestrictedDataSource deserialized = (RestrictedDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(restrictedFooDataSource), + DataSource.class + ); + + Assert.assertEquals(restrictedFooDataSource, deserialized); + } + + @Test + public void test_deserialize_fromObject() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final RestrictedDataSource deserializedRestrictedDataSource = jsonMapper.readValue( + "{\"type\":\"restrict\",\"base\":{\"type\":\"table\",\"name\":\"foo\"},\"filter\":null}", + RestrictedDataSource.class + ); + + Assert.assertEquals(restrictedFooDataSource, deserializedRestrictedDataSource); + } + + @Test + public void test_serialize() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final String s = jsonMapper.writeValueAsString(restrictedFooDataSource); + + Assert.assertEquals("{\"type\":\"restrict\",\"base\":{\"type\":\"table\",\"name\":\"foo\"},\"filter\":null}", s); + } + + @Test + public void testStringRep() + { + Assert.assertNotEquals(restrictedFooDataSource.toString(), restrictedBarDataSource.toString()); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index af17379367ed..8c6380270fa5 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -51,6 +51,9 @@ import org.apache.druid.query.Result; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.query.metadata.metadata.AggregatorMergeStrategy; import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.ListColumnIncluderator; @@ -85,6 +88,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -328,6 +332,27 @@ public void testSegmentMetadataQuery() Assert.assertEquals(Collections.singletonList(expectedSegmentAnalysis1), results); } + @Test + public void testSegmentMetadataQueryWorksWithRestrictions() throws Exception + { + ImmutableMap> noRestriction = ImmutableMap.of(DATASOURCE, Optional.empty()); + ImmutableMap> alwaysTrueRestriction = ImmutableMap.of(DATASOURCE, Optional.of( + TrueDimFilter.instance())); + ImmutableMap> withRestriction = ImmutableMap.of(DATASOURCE, Optional.of( + new NullFilter("some-column", null))); + List results1 = runner1.run(QueryPlus.wrap(testQuery.withPolicyRestrictions(noRestriction))) + .toList(); + List results2 = runner1.run(QueryPlus.wrap(testQuery.withPolicyRestrictions(alwaysTrueRestriction))) + .toList(); + + Assert.assertEquals(Collections.singletonList(expectedSegmentAnalysis1), results1); + Assert.assertEquals(Collections.singletonList(expectedSegmentAnalysis1), results2); + Assert.assertThrows( + RuntimeException.class, + () -> runner1.run(QueryPlus.wrap(testQuery.withPolicyRestrictions(withRestriction))) + ); + } + @Test public void testSegmentMetadataQueryWithRollupMerge() { diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index 99d965ec643e..bf96eec0114c 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -57,7 +57,7 @@ import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.Escalator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -975,7 +975,8 @@ public Sequence runSegmentMetadataQuery( return queryLifecycleFactory .factorize() - .runSimple(segmentMetadataQuery, escalator.createEscalatedAuthenticationResult(), Access.OK).getResults(); + .runSimple(segmentMetadataQuery, escalator.createEscalatedAuthenticationResult(), AuthorizationResult.ALLOW_ALL) + .getResults(); } @VisibleForTesting diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java index 68f264cebb48..1febc18e3150 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java @@ -142,7 +142,11 @@ private Sequence run( final TimelineLookup timelineLookup = timelineConverter.apply(timeline); QueryToolChest> toolChest = conglomerate.getToolChest(query); - Set> segmentAndServers = computeSegmentsToQuery(timelineLookup, query, toolChest); + Set> segmentAndServers = computeSegmentsToQuery( + timelineLookup, + query, + toolChest + ); queryPlus = queryPlus.withQueryMetrics(toolChest); queryPlus.getQueryMetrics().reportQueriedSegmentCount(segmentAndServers.size()).emit(emitter); @@ -181,7 +185,8 @@ Sequence getServerResults( QueryPlus queryPlus, ResponseContext responseContext, long maxQueuedBytesPerServer, - List segmentDescriptors) + List segmentDescriptors + ) { return serverRunner.run( queryPlus.withQuery( @@ -207,7 +212,10 @@ private Set> computeSegmentsToQuery List> timelineObjectHolders = intervals.stream().flatMap(i -> lookupFn.apply(i).stream()).collect(Collectors.toList()); - final List> serversLookup = toolChest.filterSegments(query, timelineObjectHolders); + final List> serversLookup = toolChest.filterSegments( + query, + timelineObjectHolders + ); Set> segmentAndServers = new HashSet<>(); for (TimelineObjectHolder holder : serversLookup) { @@ -253,7 +261,7 @@ private SortedMap> groupSegmentsByServer( private Sequence merge(Query query, List> sequencesByInterval) { return Sequences - .simple(sequencesByInterval) - .flatMerge(seq -> seq, query.getResultOrdering()); + .simple(sequencesByInterval) + .flatMerge(seq -> seq, query.getResultOrdering()); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java index 4d971db81e08..bcfff65b1852 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java @@ -19,8 +19,8 @@ package org.apache.druid.segment.realtime; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -29,6 +29,7 @@ import org.apache.druid.server.security.ResourceType; import javax.servlet.http.HttpServletRequest; +import java.util.Objects; public class ChatHandlers { @@ -37,7 +38,7 @@ public class ChatHandlers * * @return authorization result */ - public static Access authorizationCheck( + public static AuthorizationResult authorizationCheck( HttpServletRequest req, Action action, String dataSource, @@ -49,9 +50,9 @@ public static Access authorizationCheck( action ); - Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + AuthorizationResult access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); + throw new ForbiddenException(Objects.requireNonNull(access.getFailureMessage())); } return access; diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index c4ead8bedce7..5333420d6159 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -49,10 +49,10 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.server.QueryResource.ResourceIOReaderWriter; import org.apache.druid.server.log.RequestLogger; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; @@ -65,6 +65,7 @@ import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -145,7 +146,7 @@ public QueryLifecycle( public QueryResponse runSimple( final Query query, final AuthenticationResult authenticationResult, - final Access authorizationResult + final AuthorizationResult authorizationResult ) { initialize(query); @@ -156,7 +157,7 @@ public QueryResponse runSimple( try { preAuthorized(authenticationResult, authorizationResult); if (!authorizationResult.isAllowed()) { - throw new ISE(Access.DEFAULT_ERROR_MESSAGE); + throw new ISE(Objects.requireNonNull(authorizationResult.getFailureMessage())); } queryResponse = execute(); @@ -218,7 +219,7 @@ public void initialize(final Query baseQuery) * * @return authorization result */ - public Access authorize(HttpServletRequest req) + public AuthorizationResult authorize(HttpServletRequest req) { transition(State.INITIALIZED, State.AUTHORIZING); final Iterable resourcesToAuthorize = Iterables.concat( @@ -249,7 +250,7 @@ public Access authorize(HttpServletRequest req) * @param authenticationResult authentication result indicating identity of the requester * @return authorization result of requester */ - public Access authorize(AuthenticationResult authenticationResult) + public AuthorizationResult authorize(AuthenticationResult authenticationResult) { transition(State.INITIALIZED, State.AUTHORIZING); final Iterable resourcesToAuthorize = Iterables.concat( @@ -272,14 +273,14 @@ public Access authorize(AuthenticationResult authenticationResult) ); } - private void preAuthorized(final AuthenticationResult authenticationResult, final Access access) + private void preAuthorized(final AuthenticationResult authenticationResult, final AuthorizationResult authorizationResult) { // gotta transition those states, even if we are already authorized transition(State.INITIALIZED, State.AUTHORIZING); - doAuthorize(authenticationResult, access); + doAuthorize(authenticationResult, authorizationResult); } - private Access doAuthorize(final AuthenticationResult authenticationResult, final Access authorizationResult) + private AuthorizationResult doAuthorize(final AuthenticationResult authenticationResult, final AuthorizationResult authorizationResult) { Preconditions.checkNotNull(authenticationResult, "authenticationResult"); Preconditions.checkNotNull(authorizationResult, "authorizationResult"); @@ -289,6 +290,9 @@ private Access doAuthorize(final AuthenticationResult authenticationResult, fina transition(State.AUTHORIZING, State.UNAUTHORIZED); } else { transition(State.AUTHORIZING, State.AUTHORIZED); + if (!authorizationResult.equals(AuthorizationResult.ALLOW_ALL)) { + this.baseQuery = this.baseQuery.withPolicyRestrictions(authorizationResult.getPolicyFilters()); + } } this.authenticationResult = authenticationResult; @@ -455,7 +459,7 @@ public QueryToolChest getToolChest() private void transition(final State from, final State to) { if (state != from) { - throw new ISE("Cannot transition from[%s] to[%s].", from, to); + throw new ISE("Cannot transition from[%s] to[%s], current state[%s].", from, to, state); } state = to; diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 06104000b1ca..94f2d1f1f57e 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -49,8 +49,8 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.context.ResponseContext.Keys; import org.apache.druid.server.metrics.QueryCountStatsProvider; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -74,6 +74,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.Objects; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicLong; @@ -152,14 +153,14 @@ public Response cancelQuery(@PathParam("id") String queryId, @Context final Http datasources = new TreeSet<>(); } - Access authResult = AuthorizationUtils.authorizeAllResourceActions( + AuthorizationResult authResult = AuthorizationUtils.authorizeAllResourceActions( req, Iterables.transform(datasources, AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR), authorizerMapper ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } queryScheduler.cancelQuery(queryId); @@ -198,7 +199,7 @@ public Response doPost( log.debug("Got query [%s]", queryLifecycle.getQuery()); } - final Access authResult; + final AuthorizationResult authResult; try { authResult = queryLifecycle.authorize(req); } @@ -215,7 +216,7 @@ public Response doPost( } if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } final QueryResourceQueryResultPusher pusher = new QueryResourceQueryResultPusher(req, queryLifecycle, io); diff --git a/server/src/main/java/org/apache/druid/server/http/security/ConfigResourceFilter.java b/server/src/main/java/org/apache/druid/server/http/security/ConfigResourceFilter.java index 7a45ca1d5bbb..8184b165df4f 100644 --- a/server/src/main/java/org/apache/druid/server/http/security/ConfigResourceFilter.java +++ b/server/src/main/java/org/apache/druid/server/http/security/ConfigResourceFilter.java @@ -21,7 +21,7 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -29,6 +29,8 @@ import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; +import java.util.Objects; + /** * Use this ResourceFilter at end points where Druid Cluster configuration is read or written * Here are some example paths where this filter is used - @@ -56,14 +58,14 @@ public ContainerRequest filter(ContainerRequest request) getAction(request) ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return request; diff --git a/server/src/main/java/org/apache/druid/server/http/security/DatasourceResourceFilter.java b/server/src/main/java/org/apache/druid/server/http/security/DatasourceResourceFilter.java index 2e84e5bd1f38..970ba60223d8 100644 --- a/server/src/main/java/org/apache/druid/server/http/security/DatasourceResourceFilter.java +++ b/server/src/main/java/org/apache/druid/server/http/security/DatasourceResourceFilter.java @@ -23,7 +23,7 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -33,6 +33,7 @@ import javax.ws.rs.core.PathSegment; import java.util.List; +import java.util.Objects; /** * Use this resource filter for API endpoints that contain {@link #DATASOURCES_PATH_SEGMENT} in their request path. @@ -57,14 +58,14 @@ public ContainerRequest filter(ContainerRequest request) getAction(request) ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return request; diff --git a/server/src/main/java/org/apache/druid/server/http/security/RulesResourceFilter.java b/server/src/main/java/org/apache/druid/server/http/security/RulesResourceFilter.java index f314c77d7431..b90d967a91d7 100644 --- a/server/src/main/java/org/apache/druid/server/http/security/RulesResourceFilter.java +++ b/server/src/main/java/org/apache/druid/server/http/security/RulesResourceFilter.java @@ -24,7 +24,7 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -33,13 +33,14 @@ import org.apache.druid.server.security.ResourceType; import javax.ws.rs.core.PathSegment; +import java.util.Objects; /** * Use this ResourceFilter when the datasource information is present after "rules" segment in the request Path * Here are some example paths where this filter is used - - * - druid/coordinator/v1/rules/ - * */ + * - druid/coordinator/v1/rules/ + */ public class RulesResourceFilter extends AbstractResourceFilter { @@ -75,14 +76,14 @@ public boolean apply(PathSegment input) getAction(request) ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return request; diff --git a/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java index 3a2d0e3bf83c..890e37452386 100644 --- a/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java @@ -21,13 +21,15 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; +import java.util.Objects; + /** * Use this ResourceFilter at end points where Druid Cluster State is read or written * Here are some example paths where this filter is used - @@ -59,14 +61,14 @@ public ContainerRequest filter(ContainerRequest request) getAction(request) ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( + final AuthorizationResult authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } return request; diff --git a/server/src/main/java/org/apache/druid/server/security/Access.java b/server/src/main/java/org/apache/druid/server/security/Access.java index 706a78329062..7360e11ed463 100644 --- a/server/src/main/java/org/apache/druid/server/security/Access.java +++ b/server/src/main/java/org/apache/druid/server/security/Access.java @@ -21,26 +21,51 @@ import com.google.common.base.Strings; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.filter.DimFilter; + +import javax.annotation.Nullable; +import java.util.Objects; +import java.util.Optional; public class Access { public static final String DEFAULT_ERROR_MESSAGE = "Unauthorized"; + public static final String DEFAULT_AUTHORIZED_MESSAGE = "Authorized"; - public static final Access OK = new Access(true); - public static final Access DENIED = new Access(false); + public static final Access OK = Access.allow(); + public static final Access DENIED = Access.deny(""); private final boolean allowed; private final String message; + // A row-level policy filter on top of table-level read access. It should be empty if there are no policy restrictions + // or if access is requested for an action other than reading the table. + private final Optional rowFilter; public Access(boolean allowed) { - this(allowed, ""); + this(allowed, "", Optional.empty()); } - public Access(boolean allowed, String message) + Access(boolean allowed, String message, Optional rowFilter) { this.allowed = allowed; this.message = message; + this.rowFilter = rowFilter; + } + + public static Access allow() + { + return new Access(true, "", Optional.empty()); + } + + public static Access deny(@Nullable String message) + { + return new Access(false, Objects.isNull(message) ? "" : message, Optional.empty()); + } + + public static Access allowWithRestriction(Optional rowFilter) + { + return new Access(true, "", rowFilter); } public boolean isAllowed() @@ -48,25 +73,30 @@ public boolean isAllowed() return allowed; } - public String getMessage() + public Optional getRowFilter() { - return message; + return rowFilter; } - public String toMessage() + public String getMessage() { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append(allowed ? DEFAULT_AUTHORIZED_MESSAGE : DEFAULT_ERROR_MESSAGE); if (!Strings.isNullOrEmpty(message)) { - return toString(); - } else if (allowed) { - return "Authorized"; - } else { - return DEFAULT_ERROR_MESSAGE; + stringBuilder.append(", "); + stringBuilder.append(message); + } + if (allowed && rowFilter.isPresent()) { + stringBuilder.append(", with restriction "); + stringBuilder.append(rowFilter.toString()); } + return stringBuilder.toString(); } @Override public String toString() { - return StringUtils.format("Allowed:%s, Message:%s", allowed, message); + return StringUtils.format("Allowed:%s, Message:%s, Row filter: %s", allowed, message, rowFilter); } + } diff --git a/server/src/main/java/org/apache/druid/server/security/AuthorizationResult.java b/server/src/main/java/org/apache/druid/server/security/AuthorizationResult.java new file mode 100644 index 000000000000..03560fcf5a02 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/security/AuthorizationResult.java @@ -0,0 +1,115 @@ +/* + * 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.druid.server.security; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.filter.DimFilter; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +public class AuthorizationResult +{ + public static AuthorizationResult ALLOW_ALL = new AuthorizationResult(true, null, Map.of(), null, null); + public static AuthorizationResult DENY = new AuthorizationResult( + false, + Access.DENIED.getMessage(), + Map.of(), + null, + null + ); + + private final boolean isAllowed; + + @Nullable + private final String failureMessage; + + private final Map> policyFilters; + + @Nullable + private final Set sqlResourceActions; + + @Nullable + private final Set allResourceActions; + + AuthorizationResult( + boolean isAllowed, + @Nullable String failureMessage, + Map> policyFilters, + @Nullable Set sqlResourceActions, + @Nullable Set allResourceActions + ) + { + this.isAllowed = isAllowed; + this.failureMessage = failureMessage; + this.policyFilters = policyFilters; + this.sqlResourceActions = sqlResourceActions; + this.allResourceActions = allResourceActions; + } + + public static AuthorizationResult allowWithRestriction(Map> policyFilters) + { + return new AuthorizationResult(true, null, policyFilters, null, null); + } + + public AuthorizationResult withResourceActions( + Set sqlResourceActions, + Set allResourceActions + ) + { + return new AuthorizationResult( + isAllowed(), + getFailureMessage(), + ImmutableMap.copyOf(getPolicyFilters()), + sqlResourceActions, + allResourceActions + ); + } + + public boolean isAllowed() + { + return isAllowed; + } + + @Nullable + public String getFailureMessage() + { + return failureMessage; + } + + public Map> getPolicyFilters() + { + return policyFilters; + } + + @Nullable + public Set getSqlResourceActions() + { + return sqlResourceActions; + } + + @Nullable + public Set getAllResourceActions() + { + return allResourceActions; + } +} diff --git a/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java b/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java index 431819da8a42..f1834f118eff 100644 --- a/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java @@ -27,6 +27,7 @@ import org.apache.druid.audit.RequestInfo; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.filter.DimFilter; import javax.servlet.http.HttpServletRequest; import java.util.ArrayList; @@ -35,6 +36,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; /** @@ -43,21 +45,17 @@ public class AuthorizationUtils { /** - * Check a resource-action using the authorization fields from the request. - * - * Otherwise, if the resource-actions is authorized, return ACCESS_OK. - * - * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. - * - * If this attribute is already set when this function is called, an exception is thrown. + * Performs authorization check on a single resource-action based on the authentication fields from the request. + *

+ * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. If this attribute is already set + * when this function is called, an exception is thrown. * * @param request HTTP request to be authorized * @param resourceAction A resource identifier and the action to be taken the resource. * @param authorizerMapper The singleton AuthorizerMapper instance - * - * @return ACCESS_OK or the failed Access object returned by the Authorizer that checked the request. + * @return AuthorizationResult containing allow/deny access to the resource action, along with policy restrictions. */ - public static Access authorizeResourceAction( + public static AuthorizationResult authorizeResourceAction( final HttpServletRequest request, final ResourceAction resourceAction, final AuthorizerMapper authorizerMapper @@ -74,9 +72,7 @@ public static Access authorizeResourceAction( * Returns the authentication information for a request. * * @param request http request - * * @return authentication result - * * @throws IllegalStateException if the request was not authenticated */ public static AuthenticationResult authenticationResultFromRequest(final HttpServletRequest request) @@ -145,19 +141,15 @@ public static RequestInfo buildRequestInfo(String service, HttpServletRequest re } /** - * Check a list of resource-actions to be performed by the identity represented by authenticationResult. - * - * If one of the resource-actions fails the authorization check, this method returns the failed - * Access object from the check. - * - * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. + * Performs authorization check on a list of resource-actions based on the authenticationResult. + *

+ * If one of the resource-actions denys access, returns deny access immediately. * * @param authenticationResult Authentication result representing identity of requester * @param resourceActions An Iterable of resource-actions to authorize - * - * @return ACCESS_OK or the Access object from the first failed check + * @return AuthorizationResult containing allow/deny access to the resource actions, along with policy restrictions. */ - public static Access authorizeAllResourceActions( + public static AuthorizationResult authorizeAllResourceActions( final AuthenticationResult authenticationResult, final Iterable resourceActions, final AuthorizerMapper authorizerMapper @@ -170,6 +162,7 @@ public static Access authorizeAllResourceActions( // this method returns on first failure, so only successful Access results are kept in the cache final Set resultCache = new HashSet<>(); + final Map> policyFilters = new HashMap<>(); for (ResourceAction resourceAction : resourceActions) { if (resultCache.contains(resourceAction)) { @@ -181,47 +174,54 @@ public static Access authorizeAllResourceActions( resourceAction.getAction() ); if (!access.isAllowed()) { - return access; + return AuthorizationResult.DENY; } else { resultCache.add(resourceAction); + if (!resourceAction.getAction().equals(Action.READ) || !resourceAction.getResource() + .getType() + .equals(ResourceType.DATASOURCE)) { + if (access.getRowFilter().isPresent()) { + throw DruidException.defensive( + "Row policy should only present when reading a table, but was present for %s", + resourceAction + ); + } + } + policyFilters.put(resourceAction.getResource().getName(), access.getRowFilter()); } } - return Access.OK; + return AuthorizationResult.allowWithRestriction(policyFilters); } + /** - * Check a list of resource-actions to be performed as a result of an HTTP request. - * - * If one of the resource-actions fails the authorization check, this method returns the failed - * Access object from the check. - * - * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. - * - * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. - * - * If this attribute is already set when this function is called, an exception is thrown. + * Performs authorization check on a list of resource-actions based on the authentication fields from the request. + *

+ * If one of the resource-actions denys access, returns deny access immediately. + *

+ * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. If this attribute is already set + * when this function is called, an exception is thrown. * * @param request HTTP request to be authorized * @param resourceActions An Iterable of resource-actions to authorize - * - * @return ACCESS_OK or the Access object from the first failed check + * @return AuthorizationResult containing allow/deny access to the resource actions, along with policy restrictions. */ - public static Access authorizeAllResourceActions( + public static AuthorizationResult authorizeAllResourceActions( final HttpServletRequest request, final Iterable resourceActions, final AuthorizerMapper authorizerMapper ) { if (request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH) != null) { - return Access.OK; + return AuthorizationResult.ALLOW_ALL; } if (request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED) != null) { throw new ISE("Request already had authorization check."); } - Access access = authorizeAllResourceActions( + AuthorizationResult access = authorizeAllResourceActions( authenticationResultFromRequest(request), resourceActions, authorizerMapper @@ -249,28 +249,22 @@ public static void setRequestAuthorizationAttributeIfNeeded(final HttpServletReq } /** - * Filter a collection of resources by applying the resourceActionGenerator to each resource, return an iterable - * containing the filtered resources. - * - * The resourceActionGenerator returns an Iterable for each resource. - * - * If every resource-action in the iterable is authorized, the resource will be added to the filtered resources. - * - * If there is an authorization failure for one of the resource-actions, the resource will not be - * added to the returned filtered resources.. - * - * If the resourceActionGenerator returns null for a resource, that resource will not be added to the filtered - * resources. - * - * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. - * - * If this attribute is already set when this function is called, an exception is thrown. + * Return an iterable of authorized resources, by filtering the input resources with authorization checks based on the + * authentication fields from the request. This method does: + *

  • + * For every resource, resourceActionGenerator generates an Iterable of ResourceAction or null. + *
  • + * If null, continue. If any resource-action in the iterable has deny-access, cintinue. Only when every + * resource-action has allow-access, add the resource to the result. + *
  • + *

    + * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. If this attribute is already set + * when this function is called, an exception is thrown. * * @param request HTTP request to be authorized * @param resources resources to be processed into resource-actions * @param resourceActionGenerator Function that creates an iterable of resource-actions from a resource * @param authorizerMapper authorizer mapper - * * @return Iterable containing resources that were authorized */ public static Iterable filterAuthorizedResources( @@ -305,24 +299,18 @@ public static Iterable filterAuthorizedResources( } /** - * Filter a collection of resources by applying the resourceActionGenerator to each resource, return an iterable - * containing the filtered resources. - * - * The resourceActionGenerator returns an Iterable for each resource. - * - * If every resource-action in the iterable is authorized, the resource will be added to the filtered resources. - * - * If there is an authorization failure for one of the resource-actions, the resource will not be - * added to the returned filtered resources.. - * - * If the resourceActionGenerator returns null for a resource, that resource will not be added to the filtered - * resources. + * Return an iterable of authorized resources, by filtering the input resources with authorization checks based on + * authenticationResult. This method does: + *

  • + * For every resource, resourceActionGenerator generates an Iterable of ResourceAction or null. + *
  • + * If null, continue. If any resource-action in the iterable has deny-access, cintinue. Only when every + * resource-action has allow-access, add the resource to the result. * * @param authenticationResult Authentication result representing identity of requester * @param resources resources to be processed into resource-actions * @param resourceActionGenerator Function that creates an iterable of resource-actions from a resource * @param authorizerMapper authorizer mapper - * * @return Iterable containing resources that were authorized */ public static Iterable filterAuthorizedResources( @@ -369,23 +357,22 @@ public static Iterable filterAuthorizedResources( } /** - * Given a map of resource lists, filter each resources list by applying the resource action generator to each - * item in each resource list. - * - * The resourceActionGenerator returns an Iterable for each resource. - * - * If a resource list is null or has no authorized items after filtering, it will not be included in the returned - * map. - * - * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. - * - * If this attribute is already set when this function is called, an exception is thrown. + * Return a map of authorized resources, by filtering the input resources with authorization checks based on the + * authentication fields from the request. This method does: + *
  • + * For every resource, resourceActionGenerator generates an Iterable of ResourceAction or null. + *
  • + * If null, continue. If any resource-action in the iterable has deny-access, cintinue. Only when every + * resource-action has allow-access, add the resource to the result. + *
  • + *

    + * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. If this attribute is already set + * when this function is called, an exception is thrown. * * @param request HTTP request to be authorized * @param unfilteredResources Map of resource lists to be filtered * @param resourceActionGenerator Function that creates an iterable of resource-actions from a resource * @param authorizerMapper authorizer mapper - * * @return Map containing lists of resources that were authorized */ public static Map> filterAuthorizedResources( @@ -437,7 +424,7 @@ public static Map> filterAuthorizedRes * This method constructs a 'superuser' set of permissions composed of {@link Action#READ} and {@link Action#WRITE} * permissions for all known {@link ResourceType#knownTypes()} for any {@link Authorizer} implementation which is * built on pattern matching with a regex. - * + *

    * Note that if any {@link Resource} exist that use custom types not registered with * {@link ResourceType#registerResourceType}, those permissions will not be included in this list and will need to * be added manually. diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java index 22b0890e855e..17cab67c6357 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java @@ -78,8 +78,8 @@ import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -119,7 +119,8 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad { // Timeout to allow (rapid) debugging, while not blocking tests with errors. private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); - private static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create("PT1S"); + private static final SegmentMetadataCacheConfig SEGMENT_CACHE_CONFIG_DEFAULT = SegmentMetadataCacheConfig.create( + "PT1S"); private CoordinatorSegmentMetadataCache runningSchema; private CountDownLatch buildTableLatch = new CountDownLatch(1); private CountDownLatch markDataSourceLatch = new CountDownLatch(1); @@ -132,7 +133,8 @@ public void setUp() throws Exception { super.setUp(); sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); - Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(Collections.emptyList()); SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); @@ -153,7 +155,8 @@ public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch() throws Int return buildSchemaMarkAndTableLatch(SEGMENT_CACHE_CONFIG_DEFAULT); } - public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetadataCacheConfig config) throws InterruptedException + public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetadataCacheConfig config) + throws InterruptedException { Preconditions.checkState(runningSchema == null); @@ -221,7 +224,8 @@ public void testGetTableMapSomeTable() throws InterruptedException // using 'newest first' column type merge strategy, the types are expected to be the types defined in the newer // segment, except for json, which is special handled CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch( - new SegmentMetadataCacheConfig() { + new SegmentMetadataCacheConfig() + { @Override public AbstractSegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() { @@ -368,7 +372,8 @@ public void markDataSourceAsNeedRebuild(String datasource) @VisibleForTesting public void refresh( final Set segmentsToRefresh, - final Set dataSourcesToRebuild) throws IOException + final Set dataSourcesToRebuild + ) throws IOException { super.refresh(segmentsToRefresh, dataSourcesToRebuild); } @@ -379,9 +384,9 @@ public void refresh( final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); List segments = segmentMetadatas.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); Assert.assertEquals(6, segments.size()); // verify that dim3 column isn't present in schema for datasource foo @@ -446,9 +451,9 @@ public void refresh( dataSourcesToRefresh.addAll(dataSources); segments = schema.getSegmentMetadataSnapshot().values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), dataSourcesToRefresh); Assert.assertEquals(6, schema.getSegmentMetadataSnapshot().size()); @@ -503,9 +508,9 @@ public void testAvailableSegmentMetadataIsRealtime() throws InterruptedException Assert.assertEquals(1L, metadata.isRealtime()); // get the historical server final DruidServer historicalServer = druidServers.stream() - .filter(s -> s.getType().equals(ServerType.HISTORICAL)) - .findAny() - .orElse(null); + .filter(s -> s.getType().equals(ServerType.HISTORICAL)) + .findAny() + .orElse(null); Assert.assertNotNull(historicalServer); final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); @@ -523,9 +528,9 @@ public void testAvailableSegmentMetadataIsRealtime() throws InterruptedException Assert.assertEquals(0L, currentMetadata.isRealtime()); DruidServer realtimeServer = druidServers.stream() - .filter(s -> s.getType().equals(ServerType.INDEXER_EXECUTOR)) - .findAny() - .orElse(null); + .filter(s -> s.getType().equals(ServerType.INDEXER_EXECUTOR)) + .findAny() + .orElse(null); Assert.assertNotNull(realtimeServer); // drop existingSegment from realtime task schema.removeServerSegment(realtimeServer.getMetadata(), existingSegment); @@ -590,10 +595,12 @@ public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedExcep String datasource = "newSegmentAddTest"; CountDownLatch addSegmentLatch = new CountDownLatch(2); SqlSegmentsMetadataManager sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); - Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(Collections.emptyList()); SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); - Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); + Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance( + metadataManagerConfig); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -644,10 +651,12 @@ public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedEx String datasource = "newSegmentAddTest"; CountDownLatch addSegmentLatch = new CountDownLatch(1); SqlSegmentsMetadataManager sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); - Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(Collections.emptyList()); SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); - Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); + Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance( + metadataManagerConfig); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -695,10 +704,12 @@ public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedE String datasource = "newSegmentAddTest"; CountDownLatch addSegmentLatch = new CountDownLatch(1); SqlSegmentsMetadataManager sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); - Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(Collections.emptyList()); SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); - Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); + Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance( + metadataManagerConfig); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -1064,7 +1075,11 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); // This is the mat of the test, making sure that the query created by the method under test matches the expected query, specifically the operator configured context - EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + EasyMock.expect(lifecycleMock.runSimple( + expectedMetadataQuery, + AllowAllAuthenticator.ALLOW_ALL_RESULT, + AuthorizationResult.ALLOW_ALL + )) .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); EasyMock.replay(factoryMock, lifecycleMock); @@ -1174,7 +1189,11 @@ public void testSegmentMetadataFallbackType() ) ); Assert.assertEquals( - RowSignature.builder().add("a", ColumnType.STRING).add("count", ColumnType.LONG).add("distinct", ColumnType.ofComplex("hyperUnique")).build(), + RowSignature.builder() + .add("a", ColumnType.STRING) + .add("count", ColumnType.LONG) + .add("distinct", ColumnType.ofComplex("hyperUnique")) + .build(), signature ); } @@ -1266,7 +1285,10 @@ public void testMergeOrCreateRowSignatureDeltaSchema() throws InterruptedExcepti { CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE1, segment1.getId()); + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata( + DATASOURCE1, + segment1.getId() + ); Optional mergedSignature = schema.mergeOrCreateRowSignature( segment1.getId(), @@ -1300,7 +1322,10 @@ public void testMergeOrCreateRowSignatureDeltaSchemaNewUpdateColumnOldNewColumn( EmittingLogger.registerEmitter(new StubServiceEmitter("coordinator", "dummy")); - AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE1, segment1.getId()); + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata( + DATASOURCE1, + segment1.getId() + ); Optional mergedSignature = schema.mergeOrCreateRowSignature( segment1.getId(), @@ -1334,7 +1359,10 @@ public void testMergeOrCreateRowSignatureAbsoluteSchema() throws InterruptedExce { CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE1, segment1.getId()); + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata( + DATASOURCE1, + segment1.getId() + ); Optional mergedSignature = schema.mergeOrCreateRowSignature( segment1.getId(), @@ -1375,7 +1403,8 @@ public void testRealtimeSchemaAnnouncement() throws InterruptedException, IOExce backFillQueue, sqlSegmentsMetadataManager, segmentsMetadataManagerConfigSupplier - ) { + ) + { @Override void updateSchemaForRealtimeSegments(SegmentSchemas segmentSchemas) { @@ -1387,7 +1416,10 @@ void updateSchemaForRealtimeSegments(SegmentSchemas segmentSchemas) schema.onLeaderStart(); schema.awaitInitialization(); - AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE3, realtimeSegment1.getId()); + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata( + DATASOURCE3, + realtimeSegment1.getId() + ); Assert.assertNull(availableSegmentMetadata.getRowSignature()); // refresh all segments, verify that realtime segments isn't refreshed @@ -1456,7 +1488,8 @@ public void testRealtimeSchemaAnnouncementDataSourceSchemaUpdated() throws Inter backFillQueue, sqlSegmentsMetadataManager, segmentsMetadataManagerConfigSupplier - ) { + ) + { @Override public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) throws IOException @@ -1474,7 +1507,10 @@ public void refresh(Set segmentsToRefresh, Set dataSourcesToR schema.awaitInitialization(); Assert.assertTrue(refresh1Latch.await(10, TimeUnit.SECONDS)); - AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata(DATASOURCE3, realtimeSegment1.getId()); + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata( + DATASOURCE3, + realtimeSegment1.getId() + ); Assert.assertNull(availableSegmentMetadata.getRowSignature()); Assert.assertNull(schema.getDatasource(DATASOURCE3)); @@ -1585,10 +1621,18 @@ public void testSchemaBackfilling() throws InterruptedException ) )); - SegmentSchemaTestUtils segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + SegmentSchemaTestUtils segmentSchemaTestUtils = new SegmentSchemaTestUtils( + derbyConnectorRule, + derbyConnector, + mapper + ); segmentSchemaTestUtils.insertUsedSegments(segmentsToPersist, Collections.emptyMap()); - segmentSchemaManager.persistSchemaAndUpdateSegmentsTable(DATASOURCE1, pluses, CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + segmentSchemaManager.persistSchemaAndUpdateSegmentsTable( + DATASOURCE1, + pluses, + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ); ImmutableMap.Builder segmentMetadataMap = new ImmutableMap.Builder<>(); ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); @@ -1608,7 +1652,10 @@ public void testSchemaBackfilling() throws InterruptedException long numRows = r.getLong(4); SchemaPayload schemaPayload = mapper.readValue(r.getBytes(5), SchemaPayload.class); schemaPayloadMap.put(schemaFingerprint, schemaPayload); - segmentMetadataMap.put(SegmentId.tryParse(dataSource, segmentId), new SegmentMetadata(numRows, schemaFingerprint)); + segmentMetadataMap.put( + SegmentId.tryParse(dataSource, segmentId), + new SegmentMetadata(numRows, schemaFingerprint) + ); } catch (IOException e) { throw new RuntimeException(e); @@ -1638,7 +1685,8 @@ public void testSchemaBackfilling() throws InterruptedException backFillQueue, sqlSegmentsMetadataManager, segmentsMetadataManagerConfigSupplier - ) { + ) + { @Override public Set refreshSegmentsForDataSource(String dataSource, Set segments) throws IOException @@ -1728,9 +1776,9 @@ public void testSameSegmentAddedOnMultipleServer() throws InterruptedException, Map segmentsMetadata = schema.getSegmentMetadataSnapshot(); List segments = segmentsMetadata.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); Assert.assertEquals(6, segments.size()); // find the only segment with datasource "foo2" final DataSegment existingSegment = segments.stream() @@ -1771,9 +1819,9 @@ public void testSameSegmentAddedOnMultipleServer() throws InterruptedException, segmentsMetadata = schema.getSegmentMetadataSnapshot(); segments = segmentsMetadata.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); Assert.assertEquals(6, segments.size()); schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), new HashSet<>()); @@ -1833,9 +1881,9 @@ private CoordinatorSegmentMetadataCache setupForColdDatasourceSchemaTest(Service "cold-fingerprint", new SchemaPayload( RowSignature.builder() - .add("f1", ColumnType.STRING) - .add("f2", ColumnType.DOUBLE) - .build() + .add("f1", ColumnType.STRING) + .add("f2", ColumnType.DOUBLE) + .build() ) ); @@ -1905,10 +1953,26 @@ public void testColdDatasourceSchema_refreshAfterColdSchemaExec() throws IOExcep schema.coldDatasourceSchemaExec(); - emitter.verifyEmitted("metadatacache/deepStorageOnly/segment/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), 1); - emitter.verifyEmitted("metadatacache/deepStorageOnly/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), 1); - emitter.verifyEmitted("metadatacache/deepStorageOnly/segment/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), 1); - emitter.verifyEmitted("metadatacache/deepStorageOnly/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), 1); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/segment/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), + 1 + ); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/refresh/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), + 1 + ); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/segment/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), + 1 + ); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/refresh/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), + 1 + ); emitter.verifyEmitted("metadatacache/deepStorageOnly/process/time", 1); Assert.assertEquals(new HashSet<>(Arrays.asList("foo", "cold")), schema.getDataSourceInformationMap().keySet()); @@ -1986,10 +2050,26 @@ public void testColdDatasourceSchema_coldSchemaExecAfterRefresh() throws IOExcep schema.coldDatasourceSchemaExec(); - emitter.verifyEmitted("metadatacache/deepStorageOnly/segment/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), 1); - emitter.verifyEmitted("metadatacache/deepStorageOnly/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), 1); - emitter.verifyEmitted("metadatacache/deepStorageOnly/segment/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), 1); - emitter.verifyEmitted("metadatacache/deepStorageOnly/refresh/count", ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), 1); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/segment/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), + 1 + ); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/refresh/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "foo"), + 1 + ); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/segment/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), + 1 + ); + emitter.verifyEmitted( + "metadatacache/deepStorageOnly/refresh/count", + ImmutableMap.of(DruidMetrics.DATASOURCE, "cold"), + 1 + ); emitter.verifyEmitted("metadatacache/deepStorageOnly/process/time", 1); // cold datasource should be present now @@ -2038,12 +2118,12 @@ public void testColdDatasourceSchema_verifyStaleDatasourceRemoved() DataSegment coldSegmentGamma = DataSegment.builder() - .dataSource("gamma") - .interval(Intervals.of("2000/P2Y")) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(); + .dataSource("gamma") + .interval(Intervals.of("2000/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); DataSegment hotSegmentGamma = DataSegment.builder() @@ -2198,7 +2278,8 @@ public void testColdDatasourceSchemaExecRunsPeriodically() throws InterruptedExc backFillQueue, sqlSegmentsMetadataManager, segmentsMetadataManagerConfigSupplier - ) { + ) + { @Override long getColdSchemaExecPeriodMillis() { @@ -2299,7 +2380,11 @@ public void testTombstoneSegmentIsNotRefreshed() throws IOException ); EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); - EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + EasyMock.expect(lifecycleMock.runSimple( + expectedMetadataQuery, + AllowAllAuthenticator.ALLOW_ALL_RESULT, + AuthorizationResult.ALLOW_ALL + )) .andReturn(QueryResponse.withEmptyContext(Sequences.empty())).once(); EasyMock.replay(factoryMock, lifecycleMock); @@ -2342,7 +2427,8 @@ public void testUnusedSegmentIsNotRefreshed() throws InterruptedException, IOExc backFillQueue, sqlSegmentsMetadataManager, segmentsMetadataManagerConfigSupplier - ) { + ) + { @Override public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) throws IOException diff --git a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java index 8bc436ed405c..374f097fa2eb 100644 --- a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java @@ -26,38 +26,48 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.DataSource; import org.apache.druid.query.DefaultQueryConfig; import org.apache.druid.query.Druids; import org.apache.druid.query.GenericQueryMetricsFactory; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContextTest; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.RestrictedDataSource; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.server.log.RequestLogger; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceType; import org.easymock.EasyMock; +import org.easymock.IArgumentMatcher; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import javax.servlet.http.HttpServletRequest; - import java.util.HashMap; import java.util.Map; +import java.util.Optional; public class QueryLifecycleTest { @@ -157,7 +167,7 @@ public void testRunSimplePreauthorized() replayAll(); QueryLifecycle lifecycle = createLifecycle(new AuthConfig()); - lifecycle.runSimple(query, authenticationResult, Access.OK); + lifecycle.runSimple(query, authenticationResult, AuthorizationResult.ALLOW_ALL); } @Test @@ -178,7 +188,233 @@ public void testRunSimpleUnauthorized() replayAll(); QueryLifecycle lifecycle = createLifecycle(new AuthConfig()); - lifecycle.runSimple(query, authenticationResult, new Access(false)); + lifecycle.runSimple(query, authenticationResult, AuthorizationResult.DENY); + } + + @Test + public void testAuthorizedWithNoPolicyRestriction() + { + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); + EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) + .andReturn(Access.allowWithRestriction(Optional.empty())).once(); + EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject())) + .andReturn(toolChest).times(2); + EasyMock.expect(texasRanger.getQueryRunnerForIntervals( + queryMatchDataSource(TableDataSource.create(DATASOURCE)), + EasyMock.anyObject() + )) + .andReturn(runner).times(2); + EasyMock.expect(runner.run(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Sequences.empty()).times(2); + replayAll(); + + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATASOURCE) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .aggregators(new CountAggregatorFactory("chocula")) + .build(); + AuthConfig authConfig = AuthConfig.newBuilder() + .setAuthorizeQueryContextParams(true) + .build(); + QueryLifecycle lifecycle = createLifecycle(authConfig); + lifecycle.initialize(query); + Assert.assertTrue(lifecycle.authorize(authenticationResult).isAllowed()); + lifecycle.execute(); + + lifecycle = createLifecycle(authConfig); + lifecycle.runSimple(query, authenticationResult, AuthorizationResult.ALLOW_ALL); + } + + @Test + public void testAuthorizedWithAlwaysTruePolicyRestriction() + { + Optional alwaysTrueFilter = Optional.of(TrueDimFilter.instance()); + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); + EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) + .andReturn(Access.allowWithRestriction(alwaysTrueFilter)).once(); + EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject())) + .andReturn(toolChest).times(2); + EasyMock.expect(texasRanger.getQueryRunnerForIntervals(queryMatchDataSource(RestrictedDataSource.create( + TableDataSource.create(DATASOURCE), + null + )), EasyMock.anyObject())) + .andReturn(runner).times(2); + EasyMock.expect(runner.run(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Sequences.empty()).times(2); + replayAll(); + + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATASOURCE) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .aggregators(new CountAggregatorFactory("chocula")) + .build(); + AuthConfig authConfig = AuthConfig.newBuilder() + .setAuthorizeQueryContextParams(true) + .build(); + QueryLifecycle lifecycle = createLifecycle(authConfig); + lifecycle.initialize(query); + Assert.assertTrue(lifecycle.authorize(authenticationResult).isAllowed()); + lifecycle.execute(); + + lifecycle = createLifecycle(authConfig); + lifecycle.runSimple( + query, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of(DATASOURCE, alwaysTrueFilter)) + ); + } + + @Test + public void testAuthorizedWithOnePolicyRestriction() + { + Optional rowFilter = Optional.of(new NullFilter("some-column", null)); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATASOURCE) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .aggregators(new CountAggregatorFactory("chocula")) + .build(); + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); + EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) + .andReturn(Access.allowWithRestriction(rowFilter)).times(1); + EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject())) + .andReturn(toolChest).times(2); + EasyMock.expect(texasRanger.getQueryRunnerForIntervals(queryMatchDataSource(RestrictedDataSource.create( + TableDataSource.create(DATASOURCE), + rowFilter.get() + )), EasyMock.anyObject())) + .andReturn(runner).times(2); + EasyMock.expect(runner.run(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Sequences.empty()).times(2); + replayAll(); + + AuthConfig authConfig = AuthConfig.newBuilder() + .setAuthorizeQueryContextParams(true) + .build(); + QueryLifecycle lifecycle = createLifecycle(authConfig); + lifecycle.initialize(query); + Assert.assertTrue(lifecycle.authorize(authenticationResult).isAllowed()); + lifecycle.execute(); + + lifecycle = createLifecycle(authConfig); + lifecycle.runSimple( + query, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of(DATASOURCE, rowFilter)) + ); + } + + + @Test + public void testAuthorizedMissingPolicyRestriction() + { + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATASOURCE) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .aggregators(new CountAggregatorFactory("chocula")) + .build(); + final TimeseriesQuery queryOnRestrictedDS = (TimeseriesQuery) query.withPolicyRestrictions(ImmutableMap.of( + DATASOURCE, + Optional.of(TrueDimFilter.instance()) + )); + Assume.assumeTrue(queryOnRestrictedDS.getDataSource() instanceof RestrictedDataSource); + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject())).andReturn(toolChest).anyTimes(); + EasyMock.expect(toolChest.makeMetrics(EasyMock.anyObject())).andReturn(metrics).anyTimes(); + replayAll(); + + AuthConfig authConfig = AuthConfig.newBuilder() + .setAuthorizeQueryContextParams(true) + .build(); + QueryLifecycle lifecycle = createLifecycle(authConfig); + RuntimeException e = Assert.assertThrows(RuntimeException.class, () -> + lifecycle.runSimple( + query, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of()) + )); + Assert.assertEquals("Need to check row-level policy for all tables, missing [some_datasource]", e.getMessage()); + + QueryLifecycle lifecycle2 = createLifecycle(authConfig); + RuntimeException e2 = Assert.assertThrows(RuntimeException.class, () -> + lifecycle2.runSimple( + queryOnRestrictedDS, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of()) + )); + Assert.assertEquals("Missing row filter for table [some_datasource]", e2.getMessage()); + } + + @Test + public void testAuthorizedMultiplePolicyRestrictions() + { + Optional trueFilter = Optional.of(TrueDimFilter.instance()); + Optional columnFilter = Optional.of(new NullFilter("some-column", null)); + Optional columnFilter2 = Optional.of(new NullFilter("some-column2", null)); + + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(RestrictedDataSource.create( + TableDataSource.create(DATASOURCE), + columnFilter.get() + )) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .aggregators(new CountAggregatorFactory("chocula")) + .build(); + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); + EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); + EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject())).andReturn(toolChest).anyTimes(); + EasyMock.expect(toolChest.makeMetrics(EasyMock.anyObject())).andReturn(metrics).anyTimes(); + EasyMock.expect(texasRanger.getQueryRunnerForIntervals(EasyMock.anyObject(), EasyMock.anyObject())) + .andReturn(runner).times(2); + EasyMock.expect(runner.run(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Sequences.empty()).times(2); + replayAll(); + + AuthConfig authConfig = AuthConfig.newBuilder() + .setAuthorizeQueryContextParams(true) + .build(); + + QueryLifecycle lifecycle = createLifecycle(authConfig); + RuntimeException e = Assert.assertThrows(RuntimeException.class, () -> + lifecycle.runSimple( + query, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of(DATASOURCE, columnFilter2)) + )); + Assert.assertEquals( + "Incompatible restrictions on [some_datasource]: some-column IS NULL and some-column2 IS NULL", + e.getMessage() + ); + + QueryLifecycle lifecycle2 = createLifecycle(authConfig); + // trueFilter is a compatible restriction + lifecycle2.runSimple( + query, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of(DATASOURCE, trueFilter)) + ); + + lifecycle2 = createLifecycle(authConfig); + // the same filter, compatible + lifecycle2.runSimple( + query, + authenticationResult, + AuthorizationResult.allowWithRestriction(ImmutableMap.of(DATASOURCE, columnFilter)) + ); } @Test @@ -187,11 +423,23 @@ public void testAuthorizeQueryContext_authorized() EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) .andReturn(Access.OK).times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("foo", ResourceType.QUERY_CONTEXT), + Action.WRITE + )) .andReturn(Access.OK).times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("baz", ResourceType.QUERY_CONTEXT), Action.WRITE)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("baz", ResourceType.QUERY_CONTEXT), + Action.WRITE + )) .andReturn(Access.OK).times(2); EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject())) @@ -209,8 +457,8 @@ public void testAuthorizeQueryContext_authorized() .build(); AuthConfig authConfig = AuthConfig.newBuilder() - .setAuthorizeQueryContextParams(true) - .build(); + .setAuthorizeQueryContextParams(true) + .build(); QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); @@ -235,10 +483,18 @@ public void testAuthorizeQueryContext_notAuthorized() EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) .andReturn(Access.OK) .times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("foo", ResourceType.QUERY_CONTEXT), + Action.WRITE + )) .andReturn(Access.DENIED) .times(2); @@ -256,8 +512,8 @@ public void testAuthorizeQueryContext_notAuthorized() .build(); AuthConfig authConfig = AuthConfig.newBuilder() - .setAuthorizeQueryContextParams(true) - .build(); + .setAuthorizeQueryContextParams(true) + .build(); QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); Assert.assertFalse(lifecycle.authorize(mockRequest()).isAllowed()); @@ -273,7 +529,11 @@ public void testAuthorizeQueryContext_unsecuredKeys() EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) .andReturn(Access.OK) .times(2); @@ -292,9 +552,9 @@ public void testAuthorizeQueryContext_unsecuredKeys() .build(); AuthConfig authConfig = AuthConfig.newBuilder() - .setAuthorizeQueryContextParams(true) - .setUnsecuredContextKeys(ImmutableSet.of("foo", "baz")) - .build(); + .setAuthorizeQueryContextParams(true) + .setUnsecuredContextKeys(ImmutableSet.of("foo", "baz")) + .build(); QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); @@ -319,7 +579,11 @@ public void testAuthorizeQueryContext_securedKeys() EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) .andReturn(Access.OK) .times(2); @@ -338,10 +602,10 @@ public void testAuthorizeQueryContext_securedKeys() .build(); AuthConfig authConfig = AuthConfig.newBuilder() - .setAuthorizeQueryContextParams(true) - // We have secured keys, just not what the user gave. - .setSecuredContextKeys(ImmutableSet.of("foo2", "baz2")) - .build(); + .setAuthorizeQueryContextParams(true) + // We have secured keys, just not what the user gave. + .setSecuredContextKeys(ImmutableSet.of("foo2", "baz2")) + .build(); QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); @@ -366,10 +630,18 @@ public void testAuthorizeQueryContext_securedKeysNotAuthorized() EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource(DATASOURCE, ResourceType.DATASOURCE), Action.READ)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource(DATASOURCE, ResourceType.DATASOURCE), + Action.READ + )) .andReturn(Access.OK) .times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("foo", ResourceType.QUERY_CONTEXT), + Action.WRITE + )) .andReturn(Access.DENIED) .times(2); @@ -388,10 +660,10 @@ public void testAuthorizeQueryContext_securedKeysNotAuthorized() .build(); AuthConfig authConfig = AuthConfig.newBuilder() - .setAuthorizeQueryContextParams(true) - // We have secured keys. User used one of them. - .setSecuredContextKeys(ImmutableSet.of("foo", "baz2")) - .build(); + .setAuthorizeQueryContextParams(true) + // We have secured keys. User used one of them. + .setSecuredContextKeys(ImmutableSet.of("foo", "baz2")) + .build(); QueryLifecycle lifecycle = createLifecycle(authConfig); lifecycle.initialize(query); Assert.assertFalse(lifecycle.authorize(mockRequest()).isAllowed()); @@ -407,13 +679,25 @@ public void testAuthorizeLegacyQueryContext_authorized() EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); EasyMock.expect(authenticationResult.getAuthorizerName()).andReturn(AUTHORIZER).anyTimes(); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("fake", ResourceType.DATASOURCE), Action.READ)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("fake", ResourceType.DATASOURCE), + Action.READ + )) .andReturn(Access.OK) .times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("foo", ResourceType.QUERY_CONTEXT), Action.WRITE)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("foo", ResourceType.QUERY_CONTEXT), + Action.WRITE + )) .andReturn(Access.OK) .times(2); - EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("baz", ResourceType.QUERY_CONTEXT), Action.WRITE)) + EasyMock.expect(authorizer.authorize( + authenticationResult, + new Resource("baz", ResourceType.QUERY_CONTEXT), + Action.WRITE + )) .andReturn(Access.OK) .times(2); @@ -423,7 +707,12 @@ public void testAuthorizeLegacyQueryContext_authorized() replayAll(); - final QueryContextTest.LegacyContextQuery query = new QueryContextTest.LegacyContextQuery(ImmutableMap.of("foo", "bar", "baz", "qux")); + final QueryContextTest.LegacyContextQuery query = new QueryContextTest.LegacyContextQuery(ImmutableMap.of( + "foo", + "bar", + "baz", + "qux" + )); AuthConfig authConfig = AuthConfig.newBuilder() .setAuthorizeQueryContextParams(true) @@ -444,6 +733,26 @@ public void testAuthorizeLegacyQueryContext_authorized() Assert.assertTrue(lifecycle.authorize(mockRequest()).isAllowed()); } + public static Query queryMatchDataSource(DataSource dataSource) + { + EasyMock.reportMatcher(new IArgumentMatcher() + { + @Override + public boolean matches(Object query) + { + return query instanceof Query + && ((Query) query).getDataSource().equals(dataSource); + } + + @Override + public void appendTo(StringBuffer buffer) + { + buffer.append("dataSource(\"").append(dataSource).append("\")"); + } + }); + return null; + } + private HttpServletRequest mockRequest() { HttpServletRequest request = EasyMock.createNiceMock(HttpServletRequest.class); diff --git a/server/src/test/java/org/apache/druid/server/security/ForbiddenExceptionTest.java b/server/src/test/java/org/apache/druid/server/security/ForbiddenExceptionTest.java index dc3bc9144485..a818603e91ad 100644 --- a/server/src/test/java/org/apache/druid/server/security/ForbiddenExceptionTest.java +++ b/server/src/test/java/org/apache/druid/server/security/ForbiddenExceptionTest.java @@ -66,22 +66,19 @@ public void testSanitizeWithTransformFunctionReturningNewString() @Test public void testAccess() { - Access access = new Access(false); + Access access = Access.deny(null); Assert.assertFalse(access.isAllowed()); - Assert.assertEquals("", access.getMessage()); - Assert.assertEquals("Allowed:false, Message:", access.toString()); - Assert.assertEquals(Access.DEFAULT_ERROR_MESSAGE, access.toMessage()); + Assert.assertEquals("Allowed:false, Message:, Row filter: Optional.empty", access.toString()); + Assert.assertEquals(Access.DEFAULT_ERROR_MESSAGE, access.getMessage()); - access = new Access(true); + access = Access.allow(); Assert.assertTrue(access.isAllowed()); - Assert.assertEquals("", access.getMessage()); - Assert.assertEquals("Allowed:true, Message:", access.toString()); - Assert.assertEquals("Authorized", access.toMessage()); + Assert.assertEquals("Allowed:true, Message:, Row filter: Optional.empty", access.toString()); + Assert.assertEquals("Authorized", access.getMessage()); - access = new Access(false, "oops"); + access = Access.deny("oops"); Assert.assertFalse(access.isAllowed()); - Assert.assertEquals("oops", access.getMessage()); - Assert.assertEquals("Allowed:false, Message:oops", access.toString()); - Assert.assertEquals("Allowed:false, Message:oops", access.toMessage()); + Assert.assertEquals("Allowed:false, Message:oops, Row filter: Optional.empty", access.toString()); + Assert.assertEquals("Unauthorized, oops", access.getMessage()); } } diff --git a/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java b/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java index bfa95c5d5562..b3f04bf0fdda 100644 --- a/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/AbstractStatement.java @@ -21,8 +21,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryContexts; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.Resource; @@ -36,6 +36,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.function.Function; @@ -69,7 +70,7 @@ public abstract class AbstractStatement implements Closeable */ protected final Map queryContext; protected PlannerContext plannerContext; - protected DruidPlanner.AuthResult authResult; + protected AuthorizationResult authResult; protected PlannerHook hook; public AbstractStatement( @@ -137,7 +138,7 @@ protected void validate(final DruidPlanner planner) */ protected void authorize( final DruidPlanner planner, - final Function, Access> authorizer + final Function, AuthorizationResult> authorizer ) { Set securedKeys = this.sqlToolbox.plannerFactory.getAuthConfig() @@ -150,16 +151,15 @@ protected void authorize( // Authentication is done by the planner using the function provided // here. The planner ensures that this step is done before planning. authResult = planner.authorize(authorizer, contextResources); - if (!authResult.authorizationResult.isAllowed()) { - throw new ForbiddenException(authResult.authorizationResult.toMessage()); + if (!authResult.isAllowed()) { + throw new ForbiddenException(Objects.requireNonNull(authResult.getFailureMessage())); } } /** - * Resource authorizer based on the authentication result - * provided earlier. + * Returns an authorizer that can provide authorization result given a set of required resource actions and authentication result. */ - protected Function, Access> authorizer() + protected Function, AuthorizationResult> authorizer() { return resourceActions -> AuthorizationUtils.authorizeAllResourceActions( @@ -175,12 +175,12 @@ protected Function, Access> authorizer() */ public Set resources() { - return authResult.sqlResourceActions; + return Objects.requireNonNull(authResult.getSqlResourceActions()); } public Set allResources() { - return authResult.allResourceActions; + return Objects.requireNonNull(authResult.getAllResourceActions()); } public SqlQueryPlus query() diff --git a/sql/src/main/java/org/apache/druid/sql/HttpStatement.java b/sql/src/main/java/org/apache/druid/sql/HttpStatement.java index 52bef0a04f07..d02f8c6b444d 100644 --- a/sql/src/main/java/org/apache/druid/sql/HttpStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/HttpStatement.java @@ -19,7 +19,7 @@ package org.apache.druid.sql; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.http.SqlQuery; @@ -51,21 +51,21 @@ public HttpStatement( super( lifecycleToolbox, SqlQueryPlus.builder(sqlQuery) - .auth(AuthorizationUtils.authenticationResultFromRequest(req)) - .build(), + .auth(AuthorizationUtils.authenticationResultFromRequest(req)) + .build(), req.getRemoteAddr() ); this.req = req; } @Override - protected Function, Access> authorizer() + protected Function, AuthorizationResult> authorizer() { return resourceActions -> - AuthorizationUtils.authorizeAllResourceActions( - req, - resourceActions, - sqlToolbox.plannerFactory.getAuthorizerMapper() - ); + AuthorizationUtils.authorizeAllResourceActions( + req, + resourceActions, + sqlToolbox.plannerFactory.getAuthorizerMapper() + ); } } diff --git a/sql/src/main/java/org/apache/druid/sql/SqlExecutionReporter.java b/sql/src/main/java/org/apache/druid/sql/SqlExecutionReporter.java index f55f2f3f1233..224c67acfa7f 100644 --- a/sql/src/main/java/org/apache/druid/sql/SqlExecutionReporter.java +++ b/sql/src/main/java/org/apache/druid/sql/SqlExecutionReporter.java @@ -103,7 +103,7 @@ public void emit() // datasources. metricBuilder.setDimension( "dataSource", - stmt.authResult.sqlResourceActions + stmt.authResult.getSqlResourceActions() .stream() .map(action -> action.getResource().getName()) .collect(Collectors.toList()) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java index 03ef94656c5f..d236a3bc2213 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java @@ -35,7 +35,7 @@ import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.query.QueryContext; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; @@ -72,35 +72,6 @@ public enum State START, VALIDATED, PREPARED, PLANNED } - public static class AuthResult - { - public final Access authorizationResult; - - /** - * Resource actions used with authorizing a cancellation request. These actions - * include only the data-level actions (e.g. the datasource.) - */ - public final Set sqlResourceActions; - - /** - * Full resource actions authorized as part of this request. Used when logging - * resource actions. Includes query context keys, if query context authorization - * is enabled. - */ - public final Set allResourceActions; - - public AuthResult( - final Access authorizationResult, - final Set sqlResourceActions, - final Set allResourceActions - ) - { - this.authorizationResult = authorizationResult; - this.sqlResourceActions = sqlResourceActions; - this.allResourceActions = allResourceActions; - } - } - private final FrameworkConfig frameworkConfig; private final CalcitePlanner planner; private final PlannerContext plannerContext; @@ -183,7 +154,7 @@ private SqlStatementHandler createHandler(final SqlNode node) /** * Uses {@link SqlParameterizerShuttle} to rewrite {@link SqlNode} to swap out any * {@link org.apache.calcite.sql.SqlDynamicParam} early for their {@link org.apache.calcite.sql.SqlLiteral} - * replacement. + * replacement. * * @return a rewritten {@link SqlNode} with any dynamic parameters rewritten in the provided {@code original} node, * if they were present. @@ -226,14 +197,14 @@ public PrepareResult prepare() * Authorizes the statement. Done within the planner to enforce the authorization * step within the planner's state machine. * - * @param authorizer a function from resource actions to a {@link Access} result. + * @param authorizer a function produces {@link AuthorizationResult} based on resource actions. * @param extraActions set of additional resource actions beyond those inferred * from the query itself. Specifically, the set of context keys to * authorize. * @return the return value from the authorizer */ - public AuthResult authorize( - final Function, Access> authorizer, + public AuthorizationResult authorize( + final Function, AuthorizationResult> authorizer, final Set extraActions ) { @@ -241,14 +212,17 @@ public AuthResult authorize( Set sqlResourceActions = plannerContext.getResourceActions(); Set allResourceActions = new HashSet<>(sqlResourceActions); allResourceActions.addAll(extraActions); - Access access = authorizer.apply(allResourceActions); - plannerContext.setAuthorizationResult(access); + AuthorizationResult authorizationResult = authorizer.apply(allResourceActions); + plannerContext.setAuthorizationResult(authorizationResult); // Authorization is done as a flag, not a state, alas. // Views prepare without authorization, Avatica does authorize, then prepare, // so the only constraint is that authorization be done before planning. authorized = true; - return new AuthResult(access, sqlResourceActions, allResourceActions); + return authorizationResult.withResourceActions( + sqlResourceActions, + allResourceActions + ); } /** diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 51be1f93502a..c55023c5fc72 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -47,8 +47,8 @@ import org.apache.druid.query.lookup.RegisteredLookupExtractionFn; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion; @@ -146,7 +146,7 @@ public class PlannerContext // set of datasources and views which must be authorized, initialized to null so we can detect if it has been set. private Set resourceActions; // result of authorizing set of resources against authentication identity - private Access authorizationResult; + private AuthorizationResult authorizationResult; // error messages encountered while planning the query @Nullable private String planningError; @@ -411,7 +411,7 @@ public boolean isStringifyArrays() * Whether we should use {@link org.apache.druid.query.filter.BoundDimFilter} and * {@link org.apache.druid.query.filter.SelectorDimFilter} (true) or {@link org.apache.druid.query.filter.RangeFilter}, * {@link org.apache.druid.query.filter.EqualityFilter}, and {@link org.apache.druid.query.filter.NullFilter} (false). - * + *

    * Typically true when {@link NullHandling#replaceWithDefault()} and false when {@link NullHandling#sqlCompatible()}. * Can be overriden by the context parameter {@link #CTX_SQL_USE_BOUNDS_AND_SELECTORS}. */ @@ -575,7 +575,7 @@ public Object get(final String name) } - public Access getAuthorizationResult() + public AuthorizationResult getAuthorizationResult() { return Preconditions.checkNotNull(authorizationResult, "Authorization result not available"); } @@ -595,7 +595,7 @@ public void setAuthenticationResult(AuthenticationResult authenticationResult) this.authenticationResult = Preconditions.checkNotNull(authenticationResult, "authenticationResult"); } - public void setAuthorizationResult(Access access) + public void setAuthorizationResult(AuthorizationResult access) { if (this.authorizationResult != null) { // It's a bug if this happens, because setAuthorizationResult should be called exactly once. @@ -637,7 +637,7 @@ public SqlEngine getEngine() /** * Checks if the current {@link SqlEngine} supports a particular feature. - * + *

    * When executing a specific query, use this method instead of {@link SqlEngine#featureAvailable(EngineFeature)} * because it also verifies feature flags. */ diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index c21f6408b52f..eeb0dee10640 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -41,8 +41,8 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.join.JoinableFactoryWrapper; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.parser.DruidSqlParserImplFactory; @@ -127,13 +127,17 @@ public DruidPlanner createPlanner( * and ready to go authorization result. */ @VisibleForTesting - public DruidPlanner createPlannerForTesting(final SqlEngine engine, final String sql, final Map queryContext) + public DruidPlanner createPlannerForTesting( + final SqlEngine engine, + final String sql, + final Map queryContext + ) { final DruidPlanner thePlanner = createPlanner(engine, sql, queryContext, null); thePlanner.getPlannerContext() .setAuthenticationResult(NoopEscalator.getInstance().createEscalatedAuthenticationResult()); thePlanner.validate(); - thePlanner.authorize(ra -> Access.OK, ImmutableSet.of()); + thePlanner.authorize(ra -> AuthorizationResult.ALLOW_ALL, ImmutableSet.of()); return thePlanner; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java index 7b1e1ec7091d..836404319c95 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java @@ -45,8 +45,8 @@ import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryResponse; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; @@ -185,14 +185,18 @@ private QueryResponse execute( query = query.withSqlQueryId(plannerContext.getSqlQueryId()); final AuthenticationResult authenticationResult = plannerContext.getAuthenticationResult(); - final Access authorizationResult = plannerContext.getAuthorizationResult(); + final AuthorizationResult authorizationResult = plannerContext.getAuthorizationResult(); final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize(); // After calling "runSimple" the query will start running. We need to do this before reading the toolChest, since // otherwise it won't yet be initialized. (A bummer, since ideally, we'd verify the toolChest exists and can do // array-based results before starting the query; but in practice we don't expect this to happen since we keep // tight control over which query types we generate in the SQL layer. They all support array-based results.) - final QueryResponse results = queryLifecycle.runSimple((Query) query, authenticationResult, authorizationResult); + final QueryResponse results = queryLifecycle.runSimple( + (Query) query, + authenticationResult, + authorizationResult + ); return mapResultSequence( results, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 0af8c19ca064..fe2e54fef83f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -74,9 +74,9 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.metadata.AvailableSegmentMetadata; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; @@ -245,11 +245,22 @@ public SystemSchema( { Preconditions.checkNotNull(serverView, "serverView"); this.tableMap = ImmutableMap.of( - SEGMENTS_TABLE, new SegmentsTable(druidSchema, metadataView, jsonMapper, authorizerMapper), - SERVERS_TABLE, new ServersTable(druidNodeDiscoveryProvider, serverInventoryView, authorizerMapper, overlordClient, coordinatorDruidLeaderClient), - SERVER_SEGMENTS_TABLE, new ServerSegmentsTable(serverView, authorizerMapper), - TASKS_TABLE, new TasksTable(overlordClient, authorizerMapper), - SUPERVISOR_TABLE, new SupervisorsTable(overlordClient, authorizerMapper) + SEGMENTS_TABLE, + new SegmentsTable(druidSchema, metadataView, jsonMapper, authorizerMapper), + SERVERS_TABLE, + new ServersTable( + druidNodeDiscoveryProvider, + serverInventoryView, + authorizerMapper, + overlordClient, + coordinatorDruidLeaderClient + ), + SERVER_SEGMENTS_TABLE, + new ServerSegmentsTable(serverView, authorizerMapper), + TASKS_TABLE, + new TasksTable(overlordClient, authorizerMapper), + SUPERVISOR_TABLE, + new SupervisorsTable(overlordClient, authorizerMapper) ); } @@ -1136,19 +1147,20 @@ private static void checkStateReadAccessForServers( AuthorizerMapper authorizerMapper ) { - final Access stateAccess = AuthorizationUtils.authorizeAllResourceActions( + final AuthorizationResult stateAccess = AuthorizationUtils.authorizeAllResourceActions( authenticationResult, Collections.singletonList(new ResourceAction(Resource.STATE_RESOURCE, Action.READ)), authorizerMapper ); if (!stateAccess.isAllowed()) { - throw new ForbiddenException("Insufficient permission to view servers: " + stateAccess.toMessage()); + throw new ForbiddenException("Insufficient permission to view servers: " + + Objects.requireNonNull(stateAccess.getFailureMessage())); } } /** * Project a row using "projects" from {@link SegmentsTable#scan(DataContext, List, int[])}. - * + *

    * Also, fix up types so {@link ColumnType#STRING} are transformed to Strings if they aren't yet. This defers * computation of {@link ObjectMapper#writeValueAsString(Object)} or {@link Object#toString()} until we know we * actually need it. diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index d957e7155b5e..9be9c3077129 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -33,7 +33,7 @@ import org.apache.druid.server.QueryResultPusher; import org.apache.druid.server.ResponseContextConfig; import org.apache.druid.server.initialization.ServerConfig; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; @@ -140,7 +140,7 @@ public Response cancelQuery( return Response.status(Status.NOT_FOUND).build(); } - final Access access = authorizeCancellation(req, lifecycles); + final AuthorizationResult access = authorizeCancellation(req, lifecycles); if (access.isAllowed()) { // should remove only the lifecycles in the snapshot. @@ -332,11 +332,11 @@ public void writeException(Exception ex, OutputStream out) throws IOException /** * Authorize a query cancellation operation. - * + *

    * Considers only datasource and table resources; not context key resources when checking permissions. This means * that a user's permission to cancel a query depends on the datasource, not the context variables used in the query. */ - public Access authorizeCancellation(final HttpServletRequest req, final List cancelables) + public AuthorizationResult authorizeCancellation(final HttpServletRequest req, final List cancelables) { Set resources = cancelables .stream() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 02a1a3fe2816..6950393831f8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -83,7 +83,7 @@ import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.virtual.ListFilteredVirtualColumn; import org.apache.druid.server.QueryLifecycle; -import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.sql.calcite.DecoupledTestConfig.IgnoreQueriesReason; import org.apache.druid.sql.calcite.DecoupledTestConfig.QuidemTestCaseReason; import org.apache.druid.sql.calcite.NotYetSupported.Modes; @@ -5187,7 +5187,7 @@ public void testGroupByJoinAsNativeQueryWithUnoptimizedFilter(Map results = seq.toList(); Assert.assertEquals( ImmutableList.of(ResultRow.of("def")), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java index b613c602f633..9bcc55c799f2 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java @@ -69,8 +69,8 @@ import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.table.DatasourceTable; import org.apache.druid.sql.calcite.table.DruidTable; @@ -141,7 +141,10 @@ public BrokerSegmentMetadataCache buildSchemaMarkAndTableLatch() throws Interrup return buildSchemaMarkAndTableLatch(SEGMENT_CACHE_CONFIG_DEFAULT, new NoopCoordinatorClient()); } - public BrokerSegmentMetadataCache buildSchemaMarkAndTableLatch(BrokerSegmentMetadataCacheConfig config, CoordinatorClient coordinatorClient) throws InterruptedException + public BrokerSegmentMetadataCache buildSchemaMarkAndTableLatch( + BrokerSegmentMetadataCacheConfig config, + CoordinatorClient coordinatorClient + ) throws InterruptedException { Preconditions.checkState(runningSchema == null); runningSchema = new BrokerSegmentMetadataCache( @@ -203,7 +206,8 @@ public void markDataSourceAsNeedRebuild(String datasource) @VisibleForTesting public void refresh( final Set segmentsToRefresh, - final Set dataSourcesToRebuild) throws IOException + final Set dataSourcesToRebuild + ) throws IOException { super.refresh(segmentsToRefresh, dataSourcesToRebuild); refreshLatch.countDown(); @@ -226,14 +230,18 @@ public void testCoordinatorReturnsAllDSSchema() throws InterruptedException final RowSignature someDataSourceRowSignature = new QueryableIndexCursorFactory(indexAuto1).getRowSignature(); final RowSignature foo3RowSignature = new QueryableIndexCursorFactory(indexAuto2).getRowSignature(); - NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { + NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() + { @Override public ListenableFuture> fetchDataSourceInformation(Set datasources) { Map dataSourceInformationMap = new HashMap<>(); dataSourceInformationMap.put(DATASOURCE1, new DataSourceInformation(DATASOURCE1, dataSource1RowSignature)); dataSourceInformationMap.put(DATASOURCE2, new DataSourceInformation(DATASOURCE2, dataSource2RowSignature)); - dataSourceInformationMap.put(SOME_DATASOURCE, new DataSourceInformation(SOME_DATASOURCE, someDataSourceRowSignature)); + dataSourceInformationMap.put( + SOME_DATASOURCE, + new DataSourceInformation(SOME_DATASOURCE, someDataSourceRowSignature) + ); dataSourceInformationMap.put("foo3", new DataSourceInformation("foo3", foo3RowSignature)); return Futures.immediateFuture(new ArrayList<>(dataSourceInformationMap.values())); @@ -258,7 +266,12 @@ public ListenableFuture> fetchDataSourceInformation( schema.start(); schema.awaitInitialization(); final Set tableNames = schema.getDatasourceNames(); - Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE, "foo3"), tableNames); + Assert.assertEquals(ImmutableSet.of( + CalciteTests.DATASOURCE1, + CalciteTests.DATASOURCE2, + CalciteTests.SOME_DATASOURCE, + "foo3" + ), tableNames); Assert.assertEquals(dataSource1RowSignature, schema.getDatasource(DATASOURCE1).getRowSignature()); Assert.assertEquals(dataSource2RowSignature, schema.getDatasource(DATASOURCE2).getRowSignature()); @@ -277,14 +290,18 @@ public void testCoordinatorReturnsFewDSSchema() throws InterruptedException final RowSignature dataSource2RowSignature = new QueryableIndexCursorFactory(index2).getRowSignature(); final RowSignature someDataSourceRowSignature = new QueryableIndexCursorFactory(indexAuto1).getRowSignature(); - NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { + NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() + { @Override public ListenableFuture> fetchDataSourceInformation(Set datasources) { Map dataSourceInformationMap = new HashMap<>(); dataSourceInformationMap.put(DATASOURCE1, new DataSourceInformation(DATASOURCE1, dataSource1RowSignature)); dataSourceInformationMap.put(DATASOURCE2, new DataSourceInformation(DATASOURCE2, dataSource2RowSignature)); - dataSourceInformationMap.put(SOME_DATASOURCE, new DataSourceInformation(SOME_DATASOURCE, someDataSourceRowSignature)); + dataSourceInformationMap.put( + SOME_DATASOURCE, + new DataSourceInformation(SOME_DATASOURCE, someDataSourceRowSignature) + ); return Futures.immediateFuture(new ArrayList<>(dataSourceInformationMap.values())); } }; @@ -304,7 +321,11 @@ public ListenableFuture> fetchDataSourceInformation( QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); - EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + EasyMock.expect(lifecycleMock.runSimple( + expectedMetadataQuery, + AllowAllAuthenticator.ALLOW_ALL_RESULT, + AuthorizationResult.ALLOW_ALL + )) .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( @@ -335,7 +356,8 @@ public void testBrokerPollsAllDSSchema() throws InterruptedException { ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Set.class); CoordinatorClient coordinatorClient = Mockito.mock(CoordinatorClient.class); - Mockito.when(coordinatorClient.fetchDataSourceInformation(argumentCaptor.capture())).thenReturn(Futures.immediateFuture(null)); + Mockito.when(coordinatorClient.fetchDataSourceInformation(argumentCaptor.capture())) + .thenReturn(Futures.immediateFuture(null)); Set datsources = Sets.newHashSet(DATASOURCE1, DATASOURCE2, DATASOURCE3, SOME_DATASOURCE, "xyz", "coldDS"); Mockito.when(coordinatorClient.fetchDataSourcesWithUsedSegments()).thenReturn(Futures.immediateFuture(datsources)); @@ -386,7 +408,8 @@ public void testRefreshOnEachCycleCentralizedDatasourceSchemaEnabled() throws In new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), new NoopCoordinatorClient(), config - ) { + ) + { @Override public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) throws IOException @@ -425,7 +448,8 @@ public void testRefreshOnEachCycleCentralizedDatasourceSchemaDisabled() throws I new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), new NoopCoordinatorClient(), CentralizedDatasourceSchemaConfig.create() - ) { + ) + { @Override public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) throws IOException @@ -449,7 +473,11 @@ public void refresh(Set segmentsToRefresh, Set dataSourcesToR public void testGetTableMap() throws InterruptedException { BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - Assert.assertEquals(ImmutableSet.of(CalciteTests.DATASOURCE1, CalciteTests.DATASOURCE2, CalciteTests.SOME_DATASOURCE), schema.getDatasourceNames()); + Assert.assertEquals(ImmutableSet.of( + CalciteTests.DATASOURCE1, + CalciteTests.DATASOURCE2, + CalciteTests.SOME_DATASOURCE + ), schema.getDatasourceNames()); } @Test @@ -509,7 +537,8 @@ public void testGetTableMapSomeTable() throws InterruptedException // using 'newest first' column type merge strategy, the types are expected to be the types defined in the newer // segment, except for json, which is special handled BrokerSegmentMetadataCache schema = buildSchemaMarkAndTableLatch( - new BrokerSegmentMetadataCacheConfig() { + new BrokerSegmentMetadataCacheConfig() + { @Override public AbstractSegmentMetadataCache.ColumnTypeMergePolicy getMetadataColumnTypeMergePolicy() { @@ -603,6 +632,7 @@ public void testGetTableMapSomeTableLeastRestrictiveTypeMerge() throws Interrupt * This tests that {@link AvailableSegmentMetadata#getNumRows()} is correct in case * of multiple replicas i.e. when {@link AbstractSegmentMetadataCache#addSegment(DruidServerMetadata, DataSegment)} * is called more than once for same segment + * * @throws InterruptedException */ @Test @@ -720,7 +750,8 @@ public void markDataSourceAsNeedRebuild(String datasource) @VisibleForTesting public void refresh( final Set segmentsToRefresh, - final Set dataSourcesToRebuild) throws IOException + final Set dataSourcesToRebuild + ) throws IOException { super.refresh(segmentsToRefresh, dataSourcesToRebuild); } @@ -731,9 +762,9 @@ public void refresh( final Map segmentMetadatas = schema.getSegmentMetadataSnapshot(); List segments = segmentMetadatas.values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); Assert.assertEquals(6, segments.size()); // verify that dim3 column isn't present in the schema for foo @@ -769,20 +800,20 @@ public void refresh( ); QueryableIndex index = IndexBuilder.create() - .tmpDir(new File(tmpDir, "1")) - .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) - .schema( - new IncrementalIndexSchema.Builder() - .withMetrics( - new CountAggregatorFactory("cnt"), - new DoubleSumAggregatorFactory("m1", "m1"), - new HyperUniquesAggregatorFactory("unique_dim1", "dim1") - ) - .withRollup(false) - .build() - ) - .rows(rows) - .buildMMappedIndex(); + .tmpDir(new File(tmpDir, "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + new IncrementalIndexSchema.Builder() + .withMetrics( + new CountAggregatorFactory("cnt"), + new DoubleSumAggregatorFactory("m1", "m1"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + ) + .withRollup(false) + .build() + ) + .rows(rows) + .buildMMappedIndex(); walker.add(newSegment, index); serverView.addSegment(newSegment, ServerType.HISTORICAL); @@ -839,11 +870,11 @@ public void testNullAvailableSegmentMetadata() throws IOException, InterruptedEx /** * Test actions on the cache. The current design of the cache makes testing far harder * than it should be. - * + *

    * - The cache is refreshed on a schedule. * - Datasources are added to the refresh queue via an unsynchronized thread. * - The refresh loop always refreshes since one of the segments is dynamic. - * + *

    * The use of latches tries to keep things synchronized, but there are many * moving parts. A simpler technique is sorely needed. */ @@ -1038,7 +1069,11 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); // This is the mat of the test, making sure that the query created by the method under test matches the expected query, specifically the operator configured context - EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + EasyMock.expect(lifecycleMock.runSimple( + expectedMetadataQuery, + AllowAllAuthenticator.ALLOW_ALL_RESULT, + AuthorizationResult.ALLOW_ALL + )) .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); EasyMock.replay(factoryMock, lifecycleMock); @@ -1130,9 +1165,9 @@ public void testNoDatasourceSchemaWhenNoSegmentMetadata() throws InterruptedExce schema.awaitInitialization(); List segments = schema.getSegmentMetadataSnapshot().values() - .stream() - .map(AvailableSegmentMetadata::getSegment) - .collect(Collectors.toList()); + .stream() + .map(AvailableSegmentMetadata::getSegment) + .collect(Collectors.toList()); schema.refresh(segments.stream().map(DataSegment::getId).collect(Collectors.toSet()), Collections.singleton("foo")); @@ -1186,9 +1221,9 @@ public void testTombstoneSegmentIsNotRefreshed() throws IOException .build(); final ImmutableDruidServer historicalServer = druidServers.stream() - .filter(s -> s.getType().equals(ServerType.HISTORICAL)) - .findAny() - .orElse(null); + .filter(s -> s.getType().equals(ServerType.HISTORICAL)) + .findAny() + .orElse(null); Assert.assertNotNull(historicalServer); final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); @@ -1217,7 +1252,11 @@ public void testTombstoneSegmentIsNotRefreshed() throws IOException ); EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); - EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + EasyMock.expect(lifecycleMock.runSimple( + expectedMetadataQuery, + AllowAllAuthenticator.ALLOW_ALL_RESULT, + AuthorizationResult.ALLOW_ALL + )) .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); EasyMock.replay(factoryMock, lifecycleMock); diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index df1d94e6f206..194d970ed280 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -77,9 +77,9 @@ import org.apache.druid.server.mocks.MockHttpServletResponse; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; import org.apache.druid.server.scheduling.ManualQueryPrioritizationStrategy; -import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationResult; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.DirectStatement; @@ -2184,7 +2184,7 @@ private TestHttpStatement( @Override protected void authorize( DruidPlanner planner, - Function, Access> authorizer + Function, AuthorizationResult> authorizer ) { if (validateAndAuthorizeLatchSupplier.get() != null) {