diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 38b5c3a7854..8ffed2f764c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -35,8 +35,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; -import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.PlannerConfig; @@ -439,10 +437,9 @@ public class SqlBenchmark QueryContexts.VECTORIZE_KEY, vectorize, QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize ); - final AuthenticationResult authenticationResult = NoopEscalator.getInstance() - .createEscalatedAuthenticationResult(); - try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) { - final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query))); + final String sql = QUERIES.get(Integer.parseInt(query)); + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(context, sql)) { + final PlannerResult plannerResult = planner.plan(sql); final Sequence resultSequence = plannerResult.run(); final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); blackhole.consume(lastRow); @@ -458,10 +455,9 @@ public class SqlBenchmark QueryContexts.VECTORIZE_KEY, vectorize, QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize ); - final AuthenticationResult authenticationResult = NoopEscalator.getInstance() - .createEscalatedAuthenticationResult(); - try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) { - final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query))); + final String sql = QUERIES.get(Integer.parseInt(query)); + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(context, sql)) { + final PlannerResult plannerResult = planner.plan(sql); blackhole.consume(plannerResult); } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index fedd75aedac..0fbe44bbfda 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -36,8 +36,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; -import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.SqlVectorizedExpressionSanityTest; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.DruidPlanner; @@ -290,10 +288,9 @@ public class SqlExpressionBenchmark QueryContexts.VECTORIZE_KEY, vectorize, QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize ); - final AuthenticationResult authenticationResult = NoopEscalator.getInstance() - .createEscalatedAuthenticationResult(); - try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) { - final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query))); + final String sql = QUERIES.get(Integer.parseInt(query)); + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(context, sql)) { + final PlannerResult plannerResult = planner.plan(sql); final Sequence resultSequence = plannerResult.run(); final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); blackhole.consume(lastRow); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index 32fa2152ec1..dda14ff0b4b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -19,7 +19,6 @@ package org.apache.druid.benchmark.query; -import com.google.common.collect.ImmutableList; import org.apache.calcite.schema.SchemaPlus; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; @@ -40,8 +39,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; -import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerFactory; @@ -164,9 +161,7 @@ public class SqlVsNativeBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void queryPlanner(Blackhole blackhole) throws Exception { - final AuthenticationResult authResult = NoopEscalator.getInstance() - .createEscalatedAuthenticationResult(); - try (final DruidPlanner planner = plannerFactory.createPlanner(null, ImmutableList.of(), authResult)) { + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(null, sqlQuery)) { final PlannerResult plannerResult = planner.plan(sqlQuery); final Sequence resultSequence = plannerResult.run(); final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java index cb0d2e9e003..291383f7b11 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java @@ -1208,6 +1208,16 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu Action.WRITE ); + ResourceAction viewR = new ResourceAction( + new Resource(".*", ResourceType.VIEW), + Action.READ + ); + + ResourceAction viewW = new ResourceAction( + new Resource(".*", ResourceType.VIEW), + Action.WRITE + ); + ResourceAction configR = new ResourceAction( new Resource(".*", ResourceType.CONFIG), Action.READ @@ -1228,6 +1238,6 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu Action.WRITE ); - return Lists.newArrayList(datasourceR, datasourceW, configR, configW, stateR, stateW); + return Lists.newArrayList(datasourceR, datasourceW, viewR, viewW, configR, configW, stateR, stateW); } } 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 a82bbf8c4e9..5da60d4c62b 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -112,14 +112,15 @@ public class QueryLifecycle this.startNs = startNs; } + /** - * For callers where simplicity is desiredĀ over flexibility. This method does it all in one call. If the request - * is unauthorized, an IllegalStateException will be thrown. Logs and metrics are emitted when the Sequence is - * either fully iterated or throws an exception. + * For callers who have already authorized their query, and where simplicity is desired over flexibility. This method + * does it all in one call. Logs and metrics are emitted when the Sequence is either fully iterated or throws an + * exception. * - * @param query the query - * @param authenticationResult authentication result indicating identity of the requester - * @param remoteAddress remote address, for logging; or null if unknown + * @param query the query + * @param authenticationResult authentication result indicating identity of the requester + * @param authorizationResult authorization result of requester * * @return results */ @@ -127,7 +128,7 @@ public class QueryLifecycle public Sequence runSimple( final Query query, final AuthenticationResult authenticationResult, - @Nullable final String remoteAddress + final Access authorizationResult ) { initialize(query); @@ -135,8 +136,8 @@ public class QueryLifecycle final Sequence results; try { - final Access access = authorize(authenticationResult); - if (!access.isAllowed()) { + preAuthorized(authenticationResult, authorizationResult); + if (!authorizationResult.isAllowed()) { throw new ISE("Unauthorized"); } @@ -144,7 +145,7 @@ public class QueryLifecycle results = queryResponse.getResults(); } catch (Throwable e) { - emitLogsAndMetrics(e, remoteAddress, -1); + emitLogsAndMetrics(e, null, -1); throw e; } @@ -155,7 +156,7 @@ public class QueryLifecycle @Override public void after(final boolean isDone, final Throwable thrown) { - emitLogsAndMetrics(thrown, remoteAddress, -1); + emitLogsAndMetrics(thrown, null, -1); } } ); @@ -187,29 +188,6 @@ public class QueryLifecycle this.toolChest = warehouse.getToolChest(baseQuery); } - /** - * Authorize the query. Will return an Access object denoting whether the query is authorized or not. - * - * @param authenticationResult authentication result indicating the identity of the requester - * - * @return authorization result - */ - public Access authorize(final AuthenticationResult authenticationResult) - { - transition(State.INITIALIZED, State.AUTHORIZING); - return doAuthorize( - authenticationResult, - AuthorizationUtils.authorizeAllResourceActions( - authenticationResult, - Iterables.transform( - baseQuery.getDataSource().getTableNames(), - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR - ), - authorizerMapper - ) - ); - } - /** * Authorize the query. Will return an Access object denoting whether the query is authorized or not. * @@ -234,6 +212,13 @@ public class QueryLifecycle ); } + private void preAuthorized(final AuthenticationResult authenticationResult, final Access access) + { + // gotta transition those states, even if we are already authorized + transition(State.INITIALIZED, State.AUTHORIZING); + doAuthorize(authenticationResult, access); + } + private Access doAuthorize(final AuthenticationResult authenticationResult, final Access authorizationResult) { Preconditions.checkNotNull(authenticationResult, "authenticationResult"); 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 afb62eb8e3b..87194ce8d86 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 @@ -373,4 +373,12 @@ public class AuthorizationUtils new Resource(input, ResourceType.DATASOURCE), Action.WRITE ); + + /** + * Function for the pattern of generating a {@link ResourceAction} for reading from a given {@link Resource} + */ + public static final Function RESOURCE_READ_RA_GENERATOR = input -> new ResourceAction( + input, + Action.READ + ); } diff --git a/server/src/main/java/org/apache/druid/server/security/ResourceType.java b/server/src/main/java/org/apache/druid/server/security/ResourceType.java index 5debab21615..04cf43af184 100644 --- a/server/src/main/java/org/apache/druid/server/security/ResourceType.java +++ b/server/src/main/java/org/apache/druid/server/security/ResourceType.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.StringUtils; public enum ResourceType { DATASOURCE, + VIEW, CONFIG, STATE; diff --git a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java new file mode 100644 index 00000000000..975c2155b10 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java @@ -0,0 +1,186 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; +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.DefaultQueryConfig; +import org.apache.druid.query.Druids; +import org.apache.druid.query.GenericQueryMetricsFactory; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +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.AuthenticationResult; +import org.apache.druid.server.security.Authorizer; +import org.apache.druid.server.security.AuthorizerMapper; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class QueryLifecycleTest +{ + private static final String DATASOURCE = "some_datasource"; + private static final String IDENTITY = "some_identity"; + + private final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATASOURCE) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .aggregators(new CountAggregatorFactory("chocula")) + .build(); + QueryToolChestWarehouse toolChestWarehouse; + QuerySegmentWalker texasRanger; + GenericQueryMetricsFactory metricsFactory; + ServiceEmitter emitter; + RequestLogger requestLogger; + AuthorizerMapper authzMapper; + DefaultQueryConfig queryConfig; + + QueryLifecycle lifecycle; + + QueryToolChest toolChest; + QueryRunner runner; + QueryMetrics metrics; + AuthenticationResult authenticationResult; + Authorizer authorizer; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Before + public void setup() + { + toolChestWarehouse = EasyMock.createMock(QueryToolChestWarehouse.class); + texasRanger = EasyMock.createMock(QuerySegmentWalker.class); + metricsFactory = EasyMock.createMock(GenericQueryMetricsFactory.class); + emitter = EasyMock.createMock(ServiceEmitter.class); + requestLogger = EasyMock.createNiceMock(RequestLogger.class); + authzMapper = EasyMock.createMock(AuthorizerMapper.class); + queryConfig = EasyMock.createMock(DefaultQueryConfig.class); + + long nanos = System.nanoTime(); + long millis = System.currentTimeMillis(); + lifecycle = new QueryLifecycle( + toolChestWarehouse, + texasRanger, + metricsFactory, + emitter, + requestLogger, + authzMapper, + queryConfig, + millis, + nanos + ); + + toolChest = EasyMock.createMock(QueryToolChest.class); + runner = EasyMock.createMock(QueryRunner.class); + metrics = EasyMock.createNiceMock(QueryMetrics.class); + authenticationResult = EasyMock.createMock(AuthenticationResult.class); + authorizer = EasyMock.createMock(Authorizer.class); + } + + @After + public void teardown() + { + EasyMock.verify( + toolChestWarehouse, + texasRanger, + metricsFactory, + emitter, + requestLogger, + authzMapper, + queryConfig, + toolChest, + runner, + metrics, + authenticationResult, + authorizer + ); + } + + @Test + public void testRunSimplePreauthorized() + { + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); + EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) + .andReturn(toolChest) + .once(); + EasyMock.expect(texasRanger.getQueryRunnerForIntervals(EasyMock.anyObject(), EasyMock.anyObject())) + .andReturn(runner) + .once(); + EasyMock.expect(runner.run(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Sequences.empty()).once(); + + + replayAll(); + + lifecycle.runSimple(query, authenticationResult, Access.OK); + } + + @Test + public void testRunSimpleUnauthorized() + { + expectedException.expect(ISE.class); + expectedException.expectMessage("Unauthorized"); + + EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes(); + EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes(); + EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject())) + .andReturn(toolChest) + .once(); + + EasyMock.expect(toolChest.makeMetrics(EasyMock.anyObject())).andReturn(metrics).anyTimes(); + + + replayAll(); + + lifecycle.runSimple(query, authenticationResult, new Access(false)); + } + + private void replayAll() + { + EasyMock.replay( + toolChestWarehouse, + texasRanger, + metricsFactory, + emitter, + requestLogger, + authzMapper, + queryConfig, + toolChest, + runner, + metrics, + authenticationResult, + authorizer + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java b/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java index 41deb6d2582..c09de8d427a 100644 --- a/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java +++ b/sql/src/main/java/org/apache/druid/sql/SqlLifecycle.java @@ -20,7 +20,9 @@ package org.apache.druid.sql; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.calcite.avatica.remote.TypedValue; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.parser.SqlParseException; @@ -44,11 +46,13 @@ import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.server.security.Resource; import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.PlannerFactory; import org.apache.druid.sql.calcite.planner.PlannerResult; import org.apache.druid.sql.calcite.planner.PrepareResult; +import org.apache.druid.sql.calcite.planner.ValidationResult; import org.apache.druid.sql.http.SqlParameter; import org.apache.druid.sql.http.SqlQuery; @@ -61,6 +65,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * Similar to {@link org.apache.druid.server.QueryLifecycle}, this class manages the lifecycle of a SQL query. @@ -68,8 +73,8 @@ import java.util.concurrent.TimeUnit; * *
    *
  1. Initialization ({@link #initialize(String, Map)})
  2. - *
  3. Planning ({@link #plan(HttpServletRequest)} or {@link #plan(AuthenticationResult)})
  4. - *
  5. Authorization ({@link #authorize()})
  6. + *
  7. Validation and Authorization ({@link #validateAndAuthorize(HttpServletRequest)} or {@link #validateAndAuthorize(AuthenticationResult)})
  8. + *
  9. Planning ({@link #plan()})
  10. *
  11. Execution ({@link #execute()})
  12. *
  13. Logging ({@link #emitLogsAndMetrics(Throwable, String, long)})
  14. *
@@ -88,16 +93,24 @@ public class SqlLifecycle private final long startNs; private final Object lock = new Object(); + @GuardedBy("lock") private State state = State.NEW; // init during intialize + @GuardedBy("lock") private String sql; + @GuardedBy("lock") private Map queryContext; + @GuardedBy("lock") private List parameters; // init during plan - @Nullable private HttpServletRequest req; + @GuardedBy("lock") private PlannerContext plannerContext; + @GuardedBy("lock") + private ValidationResult validationResult; + @GuardedBy("lock") private PrepareResult prepareResult; + @GuardedBy("lock") private PlannerResult plannerResult; public SqlLifecycle( @@ -116,6 +129,11 @@ public class SqlLifecycle this.parameters = Collections.emptyList(); } + /** + * Initialize the query lifecycle, setting the raw string SQL, initial query context, and assign a sql query id. + * + * If successful (it will be), it will transition the lifecycle to {@link State#INITIALIZED}. + */ public String initialize(String sql, Map queryContext) { synchronized (lock) { @@ -126,6 +144,7 @@ public class SqlLifecycle } } + @GuardedBy("lock") private Map contextWithSqlId(Map queryContext) { Map newContext = new HashMap<>(); @@ -136,36 +155,155 @@ public class SqlLifecycle return newContext; } + @GuardedBy("lock") private String sqlQueryId() { return (String) this.queryContext.get(PlannerContext.CTX_SQL_QUERY_ID); } + /** + * Assign dynamic parameters to be used to substitute values during query exection. This can be performed at any + * part of the lifecycle. + */ public void setParameters(List parameters) - { - this.parameters = parameters; - } - - public PrepareResult prepare(AuthenticationResult authenticationResult) - throws ValidationException, RelConversionException, SqlParseException { synchronized (lock) { - try (DruidPlanner planner = plannerFactory.createPlanner(queryContext, parameters, authenticationResult)) { - // set planner context for logs/metrics in case something explodes early - this.plannerContext = planner.getPlannerContext(); - this.prepareResult = planner.prepare(sql); - return prepareResult; + this.parameters = parameters; + if (this.plannerContext != null) { + this.plannerContext.setParameters(parameters); } } } - private PlannerContext plan(AuthenticationResult authenticationResult) - throws RelConversionException + /** + * Validate SQL query and authorize against any datasources or views which will take part in the query. + * + * If successful, the lifecycle will first transition from {@link State#INITIALIZED} first to + * {@link State#AUTHORIZING} and then to either {@link State#AUTHORIZED} or {@link State#UNAUTHORIZED}. + */ + public void validateAndAuthorize(AuthenticationResult authenticationResult) { synchronized (lock) { - transition(State.INITIALIZED, State.PLANNED); - try (DruidPlanner planner = plannerFactory.createPlanner(queryContext, parameters, authenticationResult)) { - this.plannerContext = planner.getPlannerContext(); + if (state == State.AUTHORIZED) { + return; + } + transition(State.INITIALIZED, State.AUTHORIZING); + validate(authenticationResult); + Access access = doAuthorize( + AuthorizationUtils.authorizeAllResourceActions( + authenticationResult, + Iterables.transform(validationResult.getResources(), AuthorizationUtils.RESOURCE_READ_RA_GENERATOR), + plannerFactory.getAuthorizerMapper() + ) + ); + checkAccess(access); + } + } + + /** + * Validate SQL query and authorize against any datasources or views which the query. Like + * {@link #validateAndAuthorize(AuthenticationResult)} but for a {@link HttpServletRequest}. + * + * If successful, the lifecycle will first transition from {@link State#INITIALIZED} first to + * {@link State#AUTHORIZING} and then to either {@link State#AUTHORIZED} or {@link State#UNAUTHORIZED}. + */ + public void validateAndAuthorize(HttpServletRequest req) + { + synchronized (lock) { + transition(State.INITIALIZED, State.AUTHORIZING); + AuthenticationResult authResult = AuthorizationUtils.authenticationResultFromRequest(req); + validate(authResult); + Access access = doAuthorize( + AuthorizationUtils.authorizeAllResourceActions( + req, + Iterables.transform(validationResult.getResources(), AuthorizationUtils.RESOURCE_READ_RA_GENERATOR), + plannerFactory.getAuthorizerMapper() + ) + ); + checkAccess(access); + } + } + + @GuardedBy("lock") + private ValidationResult validate(AuthenticationResult authenticationResult) + { + try (DruidPlanner planner = plannerFactory.createPlanner(queryContext)) { + // set planner context for logs/metrics in case something explodes early + this.plannerContext = planner.getPlannerContext(); + this.plannerContext.setAuthenticationResult(authenticationResult); + // set parameters on planner context, if parameters have already been set + this.plannerContext.setParameters(parameters); + this.validationResult = planner.validate(sql); + return validationResult; + } + // we can't collapse catch clauses since SqlPlanningException has type-sensitive constructors. + catch (SqlParseException e) { + throw new SqlPlanningException(e); + } + catch (ValidationException e) { + throw new SqlPlanningException(e); + } + } + + @GuardedBy("lock") + private Access doAuthorize(final Access authorizationResult) + { + if (!authorizationResult.isAllowed()) { + // Not authorized; go straight to Jail, do not pass Go. + transition(State.AUTHORIZING, State.UNAUTHORIZED); + } else { + transition(State.AUTHORIZING, State.AUTHORIZED); + } + return authorizationResult; + } + + @GuardedBy("lock") + private void checkAccess(Access access) + { + plannerContext.setAuthorizationResult(access); + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + } + + /** + * Prepare the query lifecycle for execution, without completely planning into something that is executable, but + * including some initial parsing and validation and any dyanmic parameter type resolution, to support prepared + * statements via JDBC. + * + */ + public PrepareResult prepare() throws RelConversionException + { + synchronized (lock) { + if (state != State.AUTHORIZED) { + throw new ISE("Cannot prepare because current state[%s] is not [%s].", state, State.AUTHORIZED); + } + Preconditions.checkNotNull(plannerContext, "Cannot prepare, plannerContext is null"); + try (DruidPlanner planner = plannerFactory.createPlannerWithContext(plannerContext)) { + this.prepareResult = planner.prepare(sql); + return prepareResult; + } + // we can't collapse catch clauses since SqlPlanningException has type-sensitive constructors. + catch (SqlParseException e) { + throw new SqlPlanningException(e); + } + catch (ValidationException e) { + throw new SqlPlanningException(e); + } + } + } + + /** + * Plan the query to enable execution. + * + * If successful, the lifecycle will first transition from {@link State#AUTHORIZED} to {@link State#PLANNED}. + */ + public PlannerContext plan() throws RelConversionException + { + synchronized (lock) { + transition(State.AUTHORIZED, State.PLANNED); + Preconditions.checkNotNull(plannerContext, "Cannot plan, plannerContext is null"); + try (DruidPlanner planner = plannerFactory.createPlannerWithContext(plannerContext)) { this.plannerResult = planner.plan(sql); } // we can't collapse catch clauses since SqlPlanningException has type-sensitive constructors. @@ -179,84 +317,15 @@ public class SqlLifecycle } } - private PlannerContext plan(HttpServletRequest req) - throws RelConversionException - { - synchronized (lock) { - this.req = req; - return plan(AuthorizationUtils.authenticationResultFromRequest(req)); - } - } - - public RelDataType rowType() - { - synchronized (lock) { - return plannerResult != null ? plannerResult.rowType() : prepareResult.getRowType(); - } - } - - public Access authorize() - { - synchronized (lock) { - transition(State.PLANNED, State.AUTHORIZING); - - if (req != null) { - return doAuthorize( - AuthorizationUtils.authorizeAllResourceActions( - req, - Iterables.transform(plannerResult.datasourceNames(), AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR), - plannerFactory.getAuthorizerMapper() - ) - ); - } - - return doAuthorize( - AuthorizationUtils.authorizeAllResourceActions( - plannerContext.getAuthenticationResult(), - Iterables.transform(plannerResult.datasourceNames(), AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR), - plannerFactory.getAuthorizerMapper() - ) - ); - } - } - - private Access doAuthorize(final Access authorizationResult) - { - if (!authorizationResult.isAllowed()) { - // Not authorized; go straight to Jail, do not pass Go. - transition(State.AUTHORIZING, State.UNAUTHORIZED); - } else { - transition(State.AUTHORIZING, State.AUTHORIZED); - } - return authorizationResult; - } - - public PlannerContext planAndAuthorize(final AuthenticationResult authenticationResult) - throws RelConversionException - { - PlannerContext plannerContext = plan(authenticationResult); - Access access = authorize(); - if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); - } - return plannerContext; - } - - public PlannerContext planAndAuthorize(final HttpServletRequest req) - throws RelConversionException - { - PlannerContext plannerContext = plan(req); - Access access = authorize(); - if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); - } - return plannerContext; - } - + /** + * Execute the fully planned query. + * + * If successful, the lifecycle will first transition from {@link State#PLANNED} to {@link State#EXECUTING}. + */ public Sequence execute() { synchronized (lock) { - transition(State.AUTHORIZED, State.EXECUTING); + transition(State.PLANNED, State.EXECUTING); return plannerResult.run(); } } @@ -274,7 +343,8 @@ public class SqlLifecycle initialize(sql, queryContext); try { setParameters(SqlQuery.getParameterList(parameters)); - planAndAuthorize(authenticationResult); + validateAndAuthorize(authenticationResult); + plan(); result = execute(); } catch (Throwable e) { @@ -292,6 +362,22 @@ public class SqlLifecycle }); } + + @VisibleForTesting + public ValidationResult runAnalyzeResources(AuthenticationResult authenticationResult) + { + synchronized (lock) { + return validate(authenticationResult); + } + } + + public RelDataType rowType() + { + synchronized (lock) { + return plannerResult != null ? plannerResult.rowType() : prepareResult.getRowType(); + } + } + /** * Emit logs and metrics for this query. * @@ -326,8 +412,11 @@ public class SqlLifecycle metricBuilder.setDimension("id", plannerContext.getSqlQueryId()); metricBuilder.setDimension("nativeQueryIds", plannerContext.getNativeQueryIds().toString()); } - if (plannerResult != null) { - metricBuilder.setDimension("dataSource", plannerResult.datasourceNames().toString()); + if (validationResult != null) { + metricBuilder.setDimension( + "dataSource", + validationResult.getResources().stream().map(Resource::getName).collect(Collectors.toList()).toString() + ); } metricBuilder.setDimension("remoteAddress", StringUtils.nullToEmptyNonDruidDataString(remoteAddress)); metricBuilder.setDimension("success", String.valueOf(success)); @@ -370,6 +459,7 @@ public class SqlLifecycle } } + @GuardedBy("lock") private void transition(final State from, final State to) { if (state != from) { @@ -383,9 +473,9 @@ public class SqlLifecycle { NEW, INITIALIZED, - PLANNED, AUTHORIZING, AUTHORIZED, + PLANNED, EXECUTING, UNAUTHORIZED, DONE diff --git a/sql/src/main/java/org/apache/druid/sql/avatica/DruidStatement.java b/sql/src/main/java/org/apache/druid/sql/avatica/DruidStatement.java index b618cf6aa62..98379a3544e 100644 --- a/sql/src/main/java/org/apache/druid/sql/avatica/DruidStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/avatica/DruidStatement.java @@ -161,9 +161,9 @@ public class DruidStatement implements Closeable try { ensure(State.NEW); sqlLifecycle.initialize(query, queryContext); - + sqlLifecycle.validateAndAuthorize(authenticationResult); this.authenticationResult = authenticationResult; - PrepareResult prepareResult = sqlLifecycle.prepare(authenticationResult); + PrepareResult prepareResult = sqlLifecycle.prepare(); this.maxRowCount = maxRowCount; this.query = query; List params = new ArrayList<>(); @@ -196,7 +196,8 @@ public class DruidStatement implements Closeable ensure(State.PREPARED); try { sqlLifecycle.setParameters(parameters); - sqlLifecycle.planAndAuthorize(authenticationResult); + sqlLifecycle.validateAndAuthorize(authenticationResult); + sqlLifecycle.plan(); final Sequence baseSequence = yielderOpenCloseExecutor.submit(sqlLifecycle::execute).get(); // We can't apply limits greater than Integer.MAX_VALUE, ignore them. 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 3e156545544..772f6501285 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 @@ -23,7 +23,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import org.apache.calcite.DataContext; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.config.CalciteConnectionConfig; @@ -73,7 +72,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Properties; -import java.util.Set; public class DruidPlanner implements Closeable { @@ -92,8 +90,44 @@ public class DruidPlanner implements Closeable this.plannerContext = plannerContext; } + /** + * Validates an SQL query and collects a {@link ValidationResult} which contains a set of + * {@link org.apache.druid.server.security.Resource} corresponding to any Druid datasources or views which are taking + * part in the query + */ + public ValidationResult validate(final String sql) throws SqlParseException, ValidationException + { + reset(); + SqlNode parsed = planner.parse(sql); + if (parsed.getKind() == SqlKind.EXPLAIN) { + SqlExplain explain = (SqlExplain) parsed; + parsed = explain.getExplicandum(); + } + SqlValidator validator = getValidator(); + SqlNode validated; + try { + validated = validator.validate(rewriteDynamicParameters(parsed)); + } + catch (RuntimeException e) { + throw new ValidationException(e); + } + SqlResourceCollectorShuttle resourceCollectorShuttle = + new SqlResourceCollectorShuttle(validator, frameworkConfig.getDefaultSchema().getName()); + validated.accept(resourceCollectorShuttle); + plannerContext.setResources(resourceCollectorShuttle.getResources()); + return new ValidationResult(resourceCollectorShuttle.getResources()); + } + + /** + * Prepare an SQL query for execution, including some initial parsing and validation and any dyanmic parameter type + * resolution, to support prepared statements via JDBC. + * + * In some future this could perhaps re-use some of the work done by {@link #validate(String)} + * instead of repeating it, but that day is not today. + */ public PrepareResult prepare(final String sql) throws SqlParseException, ValidationException, RelConversionException { + reset(); SqlNode parsed = planner.parse(sql); SqlExplain explain = null; if (parsed.getKind() == SqlKind.EXPLAIN) { @@ -104,8 +138,6 @@ public class DruidPlanner implements Closeable RelRoot root = planner.rel(validated); RelDataType rowType = root.validatedRowType; - // this is sort of lame, planner won't cough up its validator, it is private and has no accessors, so make another - // one so we can get the parameter types... but i suppose beats creating our own Prepare and Planner implementations SqlValidator validator = getValidator(); RelDataType parameterTypes = validator.getParameterRowType(validator.validate(parsed)); @@ -116,9 +148,19 @@ public class DruidPlanner implements Closeable return new PrepareResult(rowType, parameterTypes); } - public PlannerResult plan(final String sql) - throws SqlParseException, ValidationException, RelConversionException + /** + * Plan an SQL query for execution, returning a {@link PlannerResult} which can be used to actually execute the query. + * + * Ideally, the query can be planned into a native Druid query, using + * {@link #planWithDruidConvention(SqlExplain, RelRoot)}, but will fall-back to + * {@link #planWithBindableConvention(SqlExplain, RelRoot)} if this is not possible. + * + * In some future this could perhaps re-use some of the work done by {@link #validate(String)} + * instead of repeating it, but that day is not today. + */ + public PlannerResult plan(final String sql) throws SqlParseException, ValidationException, RelConversionException { + reset(); SqlExplain explain = null; SqlNode parsed = planner.parse(sql); if (parsed.getKind() == SqlKind.EXPLAIN) { @@ -127,9 +169,8 @@ public class DruidPlanner implements Closeable } // the planner's type factory is not available until after parsing this.rexBuilder = new RexBuilder(planner.getTypeFactory()); + SqlNode parametized = rewriteDynamicParameters(parsed); - SqlParameterizerShuttle sshuttle = new SqlParameterizerShuttle(plannerContext); - SqlNode parametized = parsed.accept(sshuttle); final SqlNode validated = planner.validate(parametized); final RelRoot root = planner.rel(validated); @@ -159,38 +200,30 @@ public class DruidPlanner implements Closeable planner.close(); } - private SqlValidator getValidator() + /** + * While the actual query might not have changed, if the druid planner is re-used, we still have the need to reset the + * {@link #planner} since we do not re-use artifacts or keep track of state between + * {@link #validate}, {@link #prepare}, and {@link #plan} and instead repeat parsing and validation + * for each step. + * + * Currently, that state tracking is done in {@link org.apache.druid.sql.SqlLifecycle}, which will create a new + * planner for each of the corresponding steps so this isn't strictly necessary at this time, this method is here as + * much to make this situation explicit and provide context for a future refactor as anything else (and some tests + * do re-use the planner between validate, prepare, and plan, which will run into this issue). + * + * This could be improved by tying {@link org.apache.druid.sql.SqlLifecycle} and {@link DruidPlanner} states more + * closely with the state of {@link #planner}, instead of repeating parsing and validation between each of these + * steps. + */ + private void reset() { - Preconditions.checkNotNull(planner.getTypeFactory()); - - final CalciteConnectionConfig connectionConfig; - - if (frameworkConfig.getContext() != null) { - connectionConfig = frameworkConfig.getContext().unwrap(CalciteConnectionConfig.class); - } else { - Properties properties = new Properties(); - properties.setProperty( - CalciteConnectionProperty.CASE_SENSITIVE.camelName(), - String.valueOf(PlannerFactory.PARSER_CONFIG.caseSensitive()) - ); - connectionConfig = new CalciteConnectionConfigImpl(properties); - } - - Prepare.CatalogReader catalogReader = new CalciteCatalogReader( - CalciteSchema.from(frameworkConfig.getDefaultSchema().getParentSchema()), - CalciteSchema.from(frameworkConfig.getDefaultSchema()).path(null), - planner.getTypeFactory(), - connectionConfig - ); - - return SqlValidatorUtil.newValidator( - frameworkConfig.getOperatorTable(), - catalogReader, - planner.getTypeFactory(), - DruidConformance.instance() - ); + planner.close(); + planner.reset(); } + /** + * Construct a {@link PlannerResult} for a {@link RelNode} that is directly translatable to a native Druid query. + */ private PlannerResult planWithDruidConvention( final SqlExplain explain, final RelRoot root @@ -198,8 +231,7 @@ public class DruidPlanner implements Closeable { final RelNode possiblyWrappedRootRel = possiblyWrapRootWithOuterLimitFromContext(root); - RelParameterizerShuttle parametizer = new RelParameterizerShuttle(plannerContext); - RelNode parametized = possiblyWrappedRootRel.accept(parametizer); + RelNode parametized = rewriteRelDynamicParameters(possiblyWrappedRootRel); final DruidRel druidRel = (DruidRel) planner.transform( Rules.DRUID_CONVENTION_RULES, planner.getEmptyTraitSet() @@ -208,12 +240,15 @@ public class DruidPlanner implements Closeable parametized ); - final Set dataSourceNames = ImmutableSet.copyOf(druidRel.getDataSourceNames()); - if (explain != null) { - return planExplanation(druidRel, explain, dataSourceNames); + return planExplanation(druidRel, explain); } else { final Supplier> resultsSupplier = () -> { + // sanity check + Preconditions.checkState( + plannerContext.getResources().isEmpty() == druidRel.getDataSourceNames().isEmpty(), + "Authorization sanity check failed" + ); if (root.isRefTrivial()) { return druidRel.runQuery(); } else { @@ -231,10 +266,17 @@ public class DruidPlanner implements Closeable } }; - return new PlannerResult(resultsSupplier, root.validatedRowType, dataSourceNames); + return new PlannerResult(resultsSupplier, root.validatedRowType); } } + /** + * Construct a {@link PlannerResult} for a fall-back 'bindable' rel, for things that are not directly translatable + * to native Druid queries such as system tables and just a general purpose (but definitely not optimized) fall-back. + * + * See {@link #planWithDruidConvention(SqlExplain, RelRoot)} which will handle things which are directly translatable + * to native Druid queries. + */ private PlannerResult planWithBindableConvention( final SqlExplain explain, final RelRoot root @@ -242,9 +284,7 @@ public class DruidPlanner implements Closeable { BindableRel bindableRel = (BindableRel) planner.transform( Rules.BINDABLE_CONVENTION_RULES, - planner.getEmptyTraitSet() - .replace(BindableConvention.INSTANCE) - .plus(root.collation), + planner.getEmptyTraitSet().replace(BindableConvention.INSTANCE).plus(root.collation), root.rel ); @@ -265,7 +305,7 @@ public class DruidPlanner implements Closeable } if (explain != null) { - return planExplanation(bindableRel, explain, ImmutableSet.of()); + return planExplanation(bindableRel, explain); } else { final BindableRel theRel = bindableRel; final DataContext dataContext = plannerContext.createDataContext( @@ -273,8 +313,8 @@ public class DruidPlanner implements Closeable plannerContext.getParameters() ); final Supplier> resultsSupplier = () -> { - final Enumerable enumerable = theRel.bind(dataContext); - final Enumerator enumerator = enumerable.enumerator(); + final Enumerable enumerable = theRel.bind(dataContext); + final Enumerator enumerator = enumerable.enumerator(); return Sequences.withBaggage(new BaseSequence<>( new BaseSequence.IteratorMaker>() { @@ -298,17 +338,31 @@ public class DruidPlanner implements Closeable } @Override - public void cleanup(EnumeratorIterator iterFromMake) + public void cleanup(EnumeratorIterator iterFromMake) { } } ), enumerator::close); }; - return new PlannerResult(resultsSupplier, root.validatedRowType, ImmutableSet.of()); + return new PlannerResult(resultsSupplier, root.validatedRowType); } } + /** + * Construct a {@link PlannerResult} for an 'explain' query from a {@link RelNode} + */ + private PlannerResult planExplanation( + final RelNode rel, + final SqlExplain explain + ) + { + final String explanation = RelOptUtil.dumpPlan("", rel, explain.getFormat(), explain.getDetailLevel()); + final Supplier> resultsSupplier = Suppliers.ofInstance( + Sequences.simple(ImmutableList.of(new Object[]{explanation}))); + return new PlannerResult(resultsSupplier, getExplainStructType(rel.getCluster().getTypeFactory())); + } + /** * This method wraps the root with a {@link LogicalSort} that applies a limit (no ordering change). If the outer rel * is already a {@link Sort}, we can merge our outerLimit into it, similar to what is going on in @@ -357,18 +411,6 @@ public class DruidPlanner implements Closeable } } - private PlannerResult planExplanation( - final RelNode rel, - final SqlExplain explain, - final Set datasourceNames - ) - { - final String explanation = RelOptUtil.dumpPlan("", rel, explain.getFormat(), explain.getDetailLevel()); - final Supplier> resultsSupplier = Suppliers.ofInstance( - Sequences.simple(ImmutableList.of(new Object[]{explanation}))); - return new PlannerResult(resultsSupplier, getExplainStructType(rel.getCluster().getTypeFactory()), datasourceNames); - } - private static RelDataType getExplainStructType(RelDataTypeFactory typeFactory) { return typeFactory.createStructType( @@ -377,6 +419,69 @@ public class DruidPlanner implements Closeable ); } + /** + * Constructs an SQL validator, just like papa {@link #planner} uses. + */ + private SqlValidator getValidator() + { + // this is sort of lame, planner won't cough up its validator, which is nice and seeded after validating a query, + // but it is private and has no accessors, so make another one so we can get the parameter types... but i suppose + // beats creating our own Prepare and Planner implementations + Preconditions.checkNotNull(planner.getTypeFactory()); + + final CalciteConnectionConfig connectionConfig; + + if (frameworkConfig.getContext() != null) { + connectionConfig = frameworkConfig.getContext().unwrap(CalciteConnectionConfig.class); + } else { + Properties properties = new Properties(); + properties.setProperty( + CalciteConnectionProperty.CASE_SENSITIVE.camelName(), + String.valueOf(PlannerFactory.PARSER_CONFIG.caseSensitive()) + ); + connectionConfig = new CalciteConnectionConfigImpl(properties); + } + + Prepare.CatalogReader catalogReader = new CalciteCatalogReader( + CalciteSchema.from(frameworkConfig.getDefaultSchema().getParentSchema()), + CalciteSchema.from(frameworkConfig.getDefaultSchema()).path(null), + planner.getTypeFactory(), + connectionConfig + ); + + return SqlValidatorUtil.newValidator( + frameworkConfig.getOperatorTable(), + catalogReader, + planner.getTypeFactory(), + DruidConformance.instance() + ); + } + + /** + * 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 + */ + private SqlNode rewriteDynamicParameters(SqlNode parsed) + { + if (!plannerContext.getParameters().isEmpty()) { + SqlParameterizerShuttle sshuttle = new SqlParameterizerShuttle(plannerContext); + return parsed.accept(sshuttle); + } + return parsed; + } + + /** + * Fall-back dynamic parameter substitution using {@link RelParameterizerShuttle} in the event that + * {@link #rewriteDynamicParameters(SqlNode)} was unable to successfully substitute all parameter values, and will + * cause a failure if any dynamic a parameters are not bound. + */ + private RelNode rewriteRelDynamicParameters(RelNode rootRel) + { + RelParameterizerShuttle parameterizer = new RelParameterizerShuttle(plannerContext); + return rootRel.accept(parameterizer); + } + private static class EnumeratorIterator implements Iterator { private final Iterator it; 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 db8ff97dead..344023e5864 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 @@ -29,14 +29,18 @@ import org.apache.calcite.linq4j.QueryProvider; import org.apache.calcite.schema.SchemaPlus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.Resource; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CopyOnWriteArrayList; @@ -63,29 +67,30 @@ public class PlannerContext private final PlannerConfig plannerConfig; private final DateTime localNow; private final Map queryContext; - private final List parameters; - private final AuthenticationResult authenticationResult; private final String sqlQueryId; private final List nativeQueryIds = new CopyOnWriteArrayList<>(); - + // bindings for dynamic parameters to bind during planning + private List parameters = Collections.emptyList(); + // result of authentication, providing identity to authorize set of resources produced by validation + private AuthenticationResult authenticationResult; + // set of datasources and views which must be authorized + private Set resources = Collections.emptySet(); + // result of authorizing set of resources against authentication identity + private Access authorizationResult; private PlannerContext( final DruidOperatorTable operatorTable, final ExprMacroTable macroTable, final PlannerConfig plannerConfig, final DateTime localNow, - final Map queryContext, - final List parameters, - final AuthenticationResult authenticationResult + final Map queryContext ) { this.operatorTable = operatorTable; this.macroTable = macroTable; this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig"); this.queryContext = queryContext != null ? new HashMap<>(queryContext) : new HashMap<>(); - this.parameters = Preconditions.checkNotNull(parameters); this.localNow = Preconditions.checkNotNull(localNow, "localNow"); - this.authenticationResult = Preconditions.checkNotNull(authenticationResult, "authenticationResult"); String sqlQueryId = (String) this.queryContext.get(CTX_SQL_QUERY_ID); // special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle @@ -99,9 +104,7 @@ public class PlannerContext final DruidOperatorTable operatorTable, final ExprMacroTable macroTable, final PlannerConfig plannerConfig, - final Map queryContext, - final List parameters, - final AuthenticationResult authenticationResult + final Map queryContext ) { final DateTime utcNow; @@ -132,9 +135,7 @@ public class PlannerContext macroTable, plannerConfig.withOverrides(queryContext), utcNow.withZone(timeZone), - queryContext, - parameters, - authenticationResult + queryContext ); } @@ -204,8 +205,7 @@ public class PlannerContext new DateTime("1970-01-01T00:00:00.000", localNow.getZone()), localNow ).toDurationMillis(), - DataContext.Variable.TIME_ZONE.camelName, localNow.getZone().toTimeZone().clone(), - DATA_CTX_AUTHENTICATION_RESULT, authenticationResult + DataContext.Variable.TIME_ZONE.camelName, localNow.getZone().toTimeZone().clone() ); private final Map context; @@ -218,6 +218,9 @@ public class PlannerContext builder.put("?" + i, parameter.value); i++; } + if (authenticationResult != null) { + builder.put(DATA_CTX_AUTHENTICATION_RESULT, authenticationResult); + } context = builder.build(); } @@ -248,4 +251,35 @@ public class PlannerContext return new DruidDataContext(); } + + + public Access getAuthorizationResult() + { + return authorizationResult; + } + + public void setParameters(List parameters) + { + this.parameters = Preconditions.checkNotNull(parameters, "parameters"); + } + + public void setAuthenticationResult(AuthenticationResult authenticationResult) + { + this.authenticationResult = Preconditions.checkNotNull(authenticationResult, "authenticationResult"); + } + + public void setAuthorizationResult(Access access) + { + this.authorizationResult = Preconditions.checkNotNull(access, "authorizationResult"); + } + + public Set getResources() + { + return resources; + } + + public void setResources(Set resources) + { + this.resources = Preconditions.checkNotNull(resources, "resources"); + } } 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 4389040e5cc..fc584d7b41b 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 @@ -20,8 +20,8 @@ package org.apache.druid.sql.calcite.planner; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; -import org.apache.calcite.avatica.remote.TypedValue; import org.apache.calcite.avatica.util.Casing; import org.apache.calcite.avatica.util.Quoting; import org.apache.calcite.config.CalciteConnectionConfig; @@ -30,20 +30,22 @@ import org.apache.calcite.plan.Context; import org.apache.calcite.plan.ConventionTraitDef; import org.apache.calcite.rel.RelCollationTraitDef; import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.validate.SqlConformance; import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.ValidationException; import org.apache.druid.guice.annotations.Json; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.server.QueryLifecycleFactory; -import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.rel.QueryMaker; import org.apache.druid.sql.calcite.schema.DruidSchemaName; -import java.util.List; import java.util.Map; import java.util.Properties; @@ -89,21 +91,67 @@ public class PlannerFactory this.druidSchemaName = druidSchemaName; } - public DruidPlanner createPlanner( - final Map queryContext, - final List parameters, - final AuthenticationResult authenticationResult - ) + /** + * Create a Druid query planner from an initial query context + */ + public DruidPlanner createPlanner(final Map queryContext) { final PlannerContext plannerContext = PlannerContext.create( operatorTable, macroTable, plannerConfig, - queryContext, - parameters, - authenticationResult + queryContext ); final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext, jsonMapper); + final FrameworkConfig frameworkConfig = buildFrameworkConfig(plannerContext, queryMaker); + + return new DruidPlanner( + frameworkConfig, + plannerContext + ); + } + + /** + * Create a new Druid query planner, re-using a previous {@link PlannerContext} + */ + public DruidPlanner createPlannerWithContext(PlannerContext plannerContext) + { + final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext, jsonMapper); + final FrameworkConfig frameworkConfig = buildFrameworkConfig(plannerContext, queryMaker); + + return new DruidPlanner( + frameworkConfig, + plannerContext + ); + } + + + /** + * Not just visible for, but only for testing. Create a planner pre-loaded with an escalated authentication result + * and ready to go authorization result. + */ + @VisibleForTesting + public DruidPlanner createPlannerForTesting(final Map queryContext, String query) + { + DruidPlanner thePlanner = createPlanner(queryContext); + thePlanner.getPlannerContext().setAuthenticationResult(NoopEscalator.getInstance().createEscalatedAuthenticationResult()); + try { + thePlanner.validate(query); + } + catch (SqlParseException | ValidationException e) { + throw new RuntimeException(e); + } + thePlanner.getPlannerContext().setAuthorizationResult(Access.OK); + return thePlanner; + } + + public AuthorizerMapper getAuthorizerMapper() + { + return authorizerMapper; + } + + private FrameworkConfig buildFrameworkConfig(PlannerContext plannerContext, QueryMaker queryMaker) + { final SqlToRelConverter.Config sqlToRelConverterConfig = SqlToRelConverter .configBuilder() .withExpand(false) @@ -111,7 +159,7 @@ public class PlannerFactory .withTrimUnusedFields(false) .withInSubQueryThreshold(Integer.MAX_VALUE) .build(); - final FrameworkConfig frameworkConfig = Frameworks + return Frameworks .newConfigBuilder() .parserConfig(PARSER_CONFIG) .traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE) @@ -152,15 +200,5 @@ public class PlannerFactory } }) .build(); - - return new DruidPlanner( - frameworkConfig, - plannerContext - ); - } - - public AuthorizerMapper getAuthorizerMapper() - { - return authorizerMapper; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerResult.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerResult.java index d7ba599b8f8..666af5ff15a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerResult.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerResult.java @@ -20,31 +20,33 @@ package org.apache.druid.sql.calcite.planner; import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableSet; import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.java.util.common.guava.Sequence; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +/** + * The result of planning an SQL query with {@link DruidPlanner} can be run to produce query result, and also includes + * the output row type signature. + */ public class PlannerResult { private final Supplier> resultsSupplier; private final RelDataType rowType; - private final Set datasourceNames; private final AtomicBoolean didRun = new AtomicBoolean(); public PlannerResult( final Supplier> resultsSupplier, - final RelDataType rowType, - final Set datasourceNames + final RelDataType rowType ) { this.resultsSupplier = resultsSupplier; this.rowType = rowType; - this.datasourceNames = ImmutableSet.copyOf(datasourceNames); } + /** + * Run the query + */ public Sequence run() { if (!didRun.compareAndSet(false, true)) { @@ -58,9 +60,4 @@ public class PlannerResult { return rowType; } - - public Set datasourceNames() - { - return datasourceNames; - } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PrepareResult.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PrepareResult.java index 9e6b27b6cf5..c6660d8adcf 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PrepareResult.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PrepareResult.java @@ -21,6 +21,10 @@ package org.apache.druid.sql.calcite.planner; import org.apache.calcite.rel.type.RelDataType; +/** + * After preparing an SQL query with {@link DruidPlanner}, the artifacts produced are the output signature of the + * result row, and type information for any dynamic parameters which must be bound before planning the query. + */ public class PrepareResult { private final RelDataType rowType; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlResourceCollectorShuttle.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlResourceCollectorShuttle.java new file mode 100644 index 00000000000..aba6689aaf4 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlResourceCollectorShuttle.java @@ -0,0 +1,87 @@ +/* + * 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.sql.calcite.planner; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.util.SqlShuttle; +import org.apache.calcite.sql.validate.IdentifierNamespace; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorNamespace; +import org.apache.calcite.sql.validate.SqlValidatorTable; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; +import org.apache.druid.sql.calcite.schema.NamedViewSchema; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Walks an {@link SqlNode} to collect a set of {@link Resource} for {@link ResourceType#DATASOURCE} and + * {@link ResourceType#VIEW} to use for authorization during query planning. + * + * It works by looking for {@link SqlIdentifier} which corespond to a {@link IdentifierNamespace}, where + * {@link SqlValidatorNamespace} is calcite-speak for sources of data and {@link IdentifierNamespace} specifically are + * namespaces which are identified by a single variable, e.g. table names. + */ +public class SqlResourceCollectorShuttle extends SqlShuttle +{ + private final Set resources; + private final SqlValidator validator; + private final String druidSchemaName; + + public SqlResourceCollectorShuttle(SqlValidator validator, String druidSchemaName) + { + this.validator = validator; + this.resources = new HashSet<>(); + this.druidSchemaName = druidSchemaName; + } + + @Override + public SqlNode visit(SqlIdentifier id) + { + // raw tables and views and such will have a IdentifierNamespace + // since we are scoped to identifiers here, we should only pick up these + SqlValidatorNamespace namespace = validator.getNamespace(id); + if (namespace != null && namespace.isWrapperFor(IdentifierNamespace.class)) { + SqlValidatorTable validatorTable = namespace.getTable(); + // this should not probably be null if the namespace was not null, + if (validatorTable != null) { + List qualifiedNameParts = validatorTable.getQualifiedName(); + // 'schema'.'identifier' + if (qualifiedNameParts.size() == 2) { + final String schema = qualifiedNameParts.get(0); + if (druidSchemaName.equals(schema)) { + resources.add(new Resource(qualifiedNameParts.get(1), ResourceType.DATASOURCE)); + } else if (NamedViewSchema.NAME.equals(schema)) { + resources.add(new Resource(qualifiedNameParts.get(1), ResourceType.VIEW)); + } + } + } + } + return super.visit(id); + } + + public Set getResources() + { + return resources; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ValidationResult.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ValidationResult.java new file mode 100644 index 00000000000..81d9f98ae36 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ValidationResult.java @@ -0,0 +1,47 @@ +/* + * 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.sql.calcite.planner; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.server.security.Resource; + +import java.util.Set; + +/** + * If an SQL query can be validated by {@link DruidPlanner}, the resulting artifact is the set of {@link Resource} + * corresponding to the datasources and views which an authenticated request must be authorized for to process the + * query. + */ +public class ValidationResult +{ + private final Set resources; + + public ValidationResult( + final Set resources + ) + { + this.resources = ImmutableSet.copyOf(resources); + } + + public Set getResources() + { + return resources; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java index 5528d919a35..5def03293be 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java @@ -45,6 +45,7 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; @@ -142,13 +143,14 @@ public class QueryMaker query = query.withSqlQueryId(plannerContext.getSqlQueryId()); final AuthenticationResult authenticationResult = plannerContext.getAuthenticationResult(); + final Access 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 Sequence results = queryLifecycle.runSimple(query, authenticationResult, null); + final Sequence results = queryLifecycle.runSimple(query, authenticationResult, authorizationResult); //noinspection unchecked final QueryToolChest> toolChest = queryLifecycle.getToolChest(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java index bf79b491357..43e8ad7d891 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java @@ -60,6 +60,7 @@ public class DruidCalciteSchemaModule implements Module SqlBindings.addSchema(binder, NamedDruidSchema.class); SqlBindings.addSchema(binder, NamedSystemSchema.class); SqlBindings.addSchema(binder, NamedLookupSchema.class); + SqlBindings.addSchema(binder, NamedViewSchema.class); } @Provides diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 1e4321b4611..20138a401c7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -24,11 +24,9 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; @@ -61,12 +59,11 @@ import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.SegmentManager; 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.AuthenticationResult; import org.apache.druid.server.security.Escalator; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.table.DruidTable; -import org.apache.druid.sql.calcite.view.DruidViewMacro; -import org.apache.druid.sql.calcite.view.ViewManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -104,7 +101,6 @@ public class DruidSchema extends AbstractSchema private final QueryLifecycleFactory queryLifecycleFactory; private final PlannerConfig config; private final SegmentManager segmentManager; - private final ViewManager viewManager; private final JoinableFactory joinableFactory; private final ExecutorService cacheExec; private final ConcurrentMap tables; @@ -152,7 +148,6 @@ public class DruidSchema extends AbstractSchema final SegmentManager segmentManager, final JoinableFactory joinableFactory, final PlannerConfig config, - final ViewManager viewManager, final Escalator escalator ) { @@ -161,7 +156,6 @@ public class DruidSchema extends AbstractSchema this.segmentManager = segmentManager; this.joinableFactory = joinableFactory; this.config = Preconditions.checkNotNull(config, "config"); - this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager"); this.cacheExec = Execs.singleThreaded("DruidSchema-Cache-%d"); this.tables = new ConcurrentHashMap<>(); this.escalator = escalator; @@ -351,16 +345,6 @@ public class DruidSchema extends AbstractSchema return ImmutableMap.copyOf(tables); } - @Override - protected Multimap getFunctionMultimap() - { - final ImmutableMultimap.Builder builder = ImmutableMultimap.builder(); - for (Map.Entry entry : viewManager.getViews().entrySet()) { - builder.put(entry); - } - return builder.build(); - } - @VisibleForTesting void addSegment(final DruidServerMetadata server, final DataSegment segment) { @@ -689,7 +673,7 @@ public class DruidSchema extends AbstractSchema false ); - return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, authenticationResult, null); + return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, authenticationResult, Access.OK); } private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedViewSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedViewSchema.java new file mode 100644 index 00000000000..06959afa44b --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedViewSchema.java @@ -0,0 +1,47 @@ +/* + * 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.sql.calcite.schema; + +import com.google.inject.Inject; +import org.apache.calcite.schema.Schema; + +public class NamedViewSchema implements NamedSchema +{ + public static final String NAME = "view"; + private final ViewSchema viewSchema; + + @Inject + NamedViewSchema(ViewSchema viewSchema) + { + this.viewSchema = viewSchema; + } + + @Override + public String getSchemaName() + { + return NAME; + } + + @Override + public Schema getSchema() + { + return viewSchema; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/ViewSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/ViewSchema.java new file mode 100644 index 00000000000..ca53368385d --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/ViewSchema.java @@ -0,0 +1,54 @@ +/* + * 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.sql.calcite.schema; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMultimap; +import com.google.common.collect.Multimap; +import com.google.inject.Inject; +import org.apache.calcite.schema.Function; +import org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.druid.sql.calcite.view.DruidViewMacro; +import org.apache.druid.sql.calcite.view.ViewManager; + +import java.util.Map; + +public class ViewSchema extends AbstractSchema +{ + private final ViewManager viewManager; + + @Inject + public ViewSchema( + final ViewManager viewManager + ) + { + this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager"); + } + + @Override + protected Multimap getFunctionMultimap() + { + final ImmutableMultimap.Builder builder = ImmutableMultimap.builder(); + for (Map.Entry entry : viewManager.getViews().entrySet()) { + builder.put(entry); + } + return builder.build(); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java b/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java index f9c1ff98998..9df3b669423 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/view/DruidViewMacro.java @@ -28,7 +28,6 @@ import org.apache.calcite.schema.FunctionParameter; import org.apache.calcite.schema.TableMacro; import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.schema.impl.ViewTable; -import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.Escalator; import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.PlannerFactory; @@ -61,10 +60,7 @@ public class DruidViewMacro implements TableMacro public TranslatableTable apply(final List arguments) { final RelDataType rowType; - // Using an escalator here is a hack, but it's currently needed to get the row type. Ideally, some - // later refactoring would make this unnecessary, since there is no actual query going out herem. - final AuthenticationResult authResult = escalator.createEscalatedAuthenticationResult(); - try (final DruidPlanner planner = plannerFactory.createPlanner(null, ImmutableList.of(), authResult)) { + try (final DruidPlanner planner = plannerFactory.createPlanner(null)) { rowType = planner.plan(viewSql).rowType(); } 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 c1fe7d72eec..b880209da4a 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 @@ -97,8 +97,8 @@ public class SqlResource Thread.currentThread().setName(StringUtils.format("sql[%s]", sqlQueryId)); lifecycle.setParameters(sqlQuery.getParameterList()); - - final PlannerContext plannerContext = lifecycle.planAndAuthorize(req); + lifecycle.validateAndAuthorize(req); + final PlannerContext plannerContext = lifecycle.plan(); final DateTimeZone timeZone = plannerContext.getTimeZone(); // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values. diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 8cb5d353cd4..5b913aa34ae 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -67,6 +67,8 @@ import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthenticationResult; 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.sql.SqlLifecycle; import org.apache.druid.sql.SqlLifecycleFactory; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.planner.Calcites; @@ -99,6 +101,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; public class BaseCalciteQueryTest extends CalciteTestBase @@ -678,39 +681,12 @@ public class BaseCalciteQueryTest extends CalciteTestBase final ObjectMapper objectMapper ) throws Exception { - final InProcessViewManager viewManager = - new InProcessViewManager(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR, CalciteTests.DRUID_VIEW_MACRO_FACTORY); - SchemaPlus rootSchema = CalciteTests.createMockRootSchema( - conglomerate, - walker, + final SqlLifecycleFactory sqlLifecycleFactory = getSqlLifecycleFactory( plannerConfig, - viewManager, - authorizerMapper - ); - - final PlannerFactory plannerFactory = new PlannerFactory( - rootSchema, - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), operatorTable, macroTable, - plannerConfig, authorizerMapper, - objectMapper, - CalciteTests.DRUID_SCHEMA_NAME - ); - final SqlLifecycleFactory sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory); - - viewManager.createView( - plannerFactory, - "aview", - "SELECT SUBSTRING(dim1, 1, 1) AS dim1_firstchar FROM foo WHERE dim2 = 'a'" - ); - - viewManager.createView( - plannerFactory, - "bview", - "SELECT COUNT(*) FROM druid.foo\n" - + "WHERE __time >= CURRENT_TIMESTAMP + INTERVAL '1' DAY AND __time < TIMESTAMP '2002-01-01 00:00:00'" + objectMapper ); return sqlLifecycleFactory.factorize().runSimple(sql, queryContext, parameters, authenticationResult).toList(); @@ -754,6 +730,95 @@ public class BaseCalciteQueryTest extends CalciteTestBase } } + public Set analyzeResources( + PlannerConfig plannerConfig, + String sql, + AuthenticationResult authenticationResult + ) + { + SqlLifecycleFactory lifecycleFactory = getSqlLifecycleFactory( + plannerConfig, + CalciteTests.createOperatorTable(), + CalciteTests.createExprMacroTable(), + CalciteTests.TEST_AUTHORIZER_MAPPER, + CalciteTests.getJsonMapper() + ); + + SqlLifecycle lifecycle = lifecycleFactory.factorize(); + lifecycle.initialize(sql, ImmutableMap.of()); + return lifecycle.runAnalyzeResources(authenticationResult).getResources(); + } + + public SqlLifecycleFactory getSqlLifecycleFactory( + PlannerConfig plannerConfig, + DruidOperatorTable operatorTable, + ExprMacroTable macroTable, + AuthorizerMapper authorizerMapper, + ObjectMapper objectMapper + ) + { + final InProcessViewManager viewManager = + new InProcessViewManager(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR, CalciteTests.DRUID_VIEW_MACRO_FACTORY); + SchemaPlus rootSchema = CalciteTests.createMockRootSchema( + conglomerate, + walker, + plannerConfig, + viewManager, + authorizerMapper + ); + + final PlannerFactory plannerFactory = new PlannerFactory( + rootSchema, + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + operatorTable, + macroTable, + plannerConfig, + authorizerMapper, + objectMapper, + CalciteTests.DRUID_SCHEMA_NAME + ); + final SqlLifecycleFactory sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory); + + viewManager.createView( + plannerFactory, + "aview", + "SELECT SUBSTRING(dim1, 1, 1) AS dim1_firstchar FROM foo WHERE dim2 = 'a'" + ); + + viewManager.createView( + plannerFactory, + "bview", + "SELECT COUNT(*) FROM druid.foo\n" + + "WHERE __time >= CURRENT_TIMESTAMP + INTERVAL '1' DAY AND __time < TIMESTAMP '2002-01-01 00:00:00'" + ); + + viewManager.createView( + plannerFactory, + "cview", + "SELECT SUBSTRING(bar.dim1, 1, 1) AS dim1_firstchar, bar.dim2 as dim2, dnf.l2 as l2\n" + + "FROM (SELECT * from foo WHERE dim2 = 'a') as bar INNER JOIN druid.numfoo dnf ON bar.dim2 = dnf.dim2" + ); + + viewManager.createView( + plannerFactory, + "dview", + "SELECT SUBSTRING(dim1, 1, 1) AS numfoo FROM foo WHERE dim2 = 'a'" + ); + + viewManager.createView( + plannerFactory, + "forbiddenView", + "SELECT __time, SUBSTRING(dim1, 1, 1) AS dim1_firstchar, dim2 FROM foo WHERE dim2 = 'a'" + ); + + viewManager.createView( + plannerFactory, + "restrictedView", + "SELECT __time, dim1, dim2, m1 FROM druid.forbiddenDatasource WHERE dim2 = 'a'" + ); + return sqlLifecycleFactory; + } + protected void cannotVectorize() { cannotVectorize = true; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 6b35a8dbb7e..f4a941b6de0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -105,6 +105,7 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.join.JoinType; import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.security.Access; import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.SqlPlanningException.PlanningError; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -793,6 +794,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ImmutableList.of(), ImmutableList.of( new Object[]{"lookup"}, + new Object[]{"view"}, new Object[]{"druid"}, new Object[]{"sys"}, new Object[]{"INFORMATION_SCHEMA"} @@ -817,8 +819,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE", "NO", "NO"}) .add(new Object[]{"druid", CalciteTests.SOME_DATASOURCE, "TABLE", "NO", "NO"}) .add(new Object[]{"druid", CalciteTests.SOMEXDATASOURCE, "TABLE", "NO", "NO"}) - .add(new Object[]{"druid", "aview", "VIEW", "NO", "NO"}) - .add(new Object[]{"druid", "bview", "VIEW", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE", "NO", "NO"}) @@ -828,6 +828,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .add(new Object[]{"sys", "servers", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "supervisors", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"view", "aview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "bview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "cview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "dview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "forbiddenView", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "restrictedView", "VIEW", "NO", "NO"}) .build() ); @@ -848,8 +854,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE", "NO", "NO"}) .add(new Object[]{"druid", CalciteTests.SOME_DATASOURCE, "TABLE", "NO", "NO"}) .add(new Object[]{"druid", CalciteTests.SOMEXDATASOURCE, "TABLE", "NO", "NO"}) - .add(new Object[]{"druid", "aview", "VIEW", "NO", "NO"}) - .add(new Object[]{"druid", "bview", "VIEW", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE", "NO", "NO"}) @@ -859,6 +863,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .add(new Object[]{"sys", "servers", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "supervisors", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"view", "aview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "bview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "cview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "dview", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "forbiddenView", "VIEW", "NO", "NO"}) + .add(new Object[]{"view", "restrictedView", "VIEW", "NO", "NO"}) .build() ); } @@ -920,7 +930,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" - + "WHERE TABLE_SCHEMA = 'druid' AND TABLE_NAME = 'aview'", + + "WHERE TABLE_SCHEMA = 'view' AND TABLE_NAME = 'aview'", ImmutableList.of(), ImmutableList.of( new Object[]{"dim1_firstchar", "VARCHAR", "YES"} @@ -928,6 +938,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testInformationSchemaColumnsOnAnotherView() throws Exception + { + testQuery( + "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + + "FROM INFORMATION_SCHEMA.COLUMNS\n" + + "WHERE TABLE_SCHEMA = 'view' AND TABLE_NAME = 'cview'", + ImmutableList.of(), + ImmutableList.of( + new Object[]{"dim1_firstchar", "VARCHAR", "YES"}, + new Object[]{"dim2", "VARCHAR", "YES"}, + new Object[]{"l2", "BIGINT", useDefault ? "NO" : "YES"} + ) + ); + } + @Test public void testExplainInformationSchemaColumns() throws Exception { @@ -1026,6 +1052,108 @@ public class CalciteQueryTest extends BaseCalciteQueryTest 9999.0f, NullHandling.defaultDoubleValue(), "\"AQAAAQAAAALFBA==\"" + }, + new Object[]{ + timestamp("2000-01-02"), + 1L, + "forbidden", + "a", + 1234.0f, + NullHandling.defaultDoubleValue(), + "\"AQAAAQAAAALFBA==\"" + } + ) + ); + } + + @Test + public void testSelectStarOnForbiddenView() throws Exception + { + assertQueryIsForbidden( + "SELECT * FROM view.forbiddenView", + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + testQuery( + PLANNER_CONFIG_DEFAULT, + "SELECT * FROM view.forbiddenView", + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ValueType.STRING), + expressionVirtualColumn("v1", "'a'", ValueType.STRING) + ) + .filters(selector("dim2", "a", null)) + .columns("__time", "v0", "v1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{ + timestamp("2000-01-01"), + NullHandling.defaultStringValue(), + "a" + }, + new Object[]{ + timestamp("2001-01-01"), + "1", + "a" + } + ) + ); + } + + @Test + public void testSelectStarOnRestrictedView() throws Exception + { + testQuery( + PLANNER_CONFIG_DEFAULT, + "SELECT * FROM view.restrictedView", + CalciteTests.REGULAR_USER_AUTH_RESULT, + ImmutableList.of( + newScanQueryBuilder() + .dataSource(CalciteTests.FORBIDDEN_DATASOURCE) + .filters(selector("dim2", "a", null)) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("__time", "dim1", "dim2", "m1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{ + timestamp("2000-01-02"), + "forbidden", + "a", + 1234.0f + } + ) + ); + + testQuery( + PLANNER_CONFIG_DEFAULT, + "SELECT * FROM view.restrictedView", + CalciteTests.SUPER_USER_AUTH_RESULT, + ImmutableList.of( + newScanQueryBuilder() + .dataSource(CalciteTests.FORBIDDEN_DATASOURCE) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(selector("dim2", "a", null)) + .columns("__time", "dim1", "dim2", "m1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{ + timestamp("2000-01-02"), + "forbidden", + "a", + 1234.0f } ) ); @@ -5125,7 +5253,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testCountStarOnView() throws Exception { testQuery( - "SELECT COUNT(*) FROM druid.aview WHERE dim1_firstchar <> 'z'", + "SELECT COUNT(*) FROM view.aview WHERE dim1_firstchar <> 'z'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -5145,6 +5273,85 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testConfusedView() throws Exception + { + testQuery( + "SELECT COUNT(*) FROM view.dview as druid WHERE druid.numfoo <> 'z'", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(and( + selector("dim2", "a", null), + not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))) + )) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{2L} + ) + ); + } + + @Test + public void testViewAndJoin() throws Exception + { + cannotVectorize(); + testQuery( + "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource( + join( + join( + new QueryDataSource( + newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim1", "dim2") + .filters(selector("dim2", "a", null)) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + new QueryDataSource( + newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "(\"dim2\" == \"j0.dim2\")", + JoinType.INNER + ), + new QueryDataSource( + newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "_j0.", + "('a' == \"_j0.dim2\")", + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1)))) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{8L} + ) + ); + } + @Test public void testExplainCountStarOnView() throws Exception { @@ -5167,7 +5374,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + ", signature=[{a0:LONG}])\n"; testQuery( - "EXPLAIN PLAN FOR SELECT COUNT(*) FROM aview WHERE dim1_firstchar <> 'z'", + "EXPLAIN PLAN FOR SELECT COUNT(*) FROM view.aview WHERE dim1_firstchar <> 'z'", ImmutableList.of(), ImmutableList.of( new Object[]{explanation} @@ -8829,7 +9036,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testFilterOnCurrentTimestampOnView() throws Exception { testQuery( - "SELECT * FROM bview", + "SELECT * FROM view.bview", ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -8854,7 +9061,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery( PLANNER_CONFIG_DEFAULT, QUERY_CONTEXT_LOS_ANGELES, - "SELECT * FROM bview", + "SELECT * FROM view.bview", CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of( Druids.newTimeseriesQueryBuilder() @@ -13074,11 +13281,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest QueryLifecycleFactory qlf = CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate); QueryLifecycle ql = qlf.factorize(); - Sequence seq = ql.runSimple( - query, - CalciteTests.SUPER_USER_AUTH_RESULT, - null - ); + Sequence seq = ql.runSimple(query, CalciteTests.SUPER_USER_AUTH_RESULT, Access.OK); List results = seq.toList(); Assert.assertEquals( ImmutableList.of(ResultRow.of("def")), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DruidPlannerResourceAnalyzeTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DruidPlannerResourceAnalyzeTest.java new file mode 100644 index 00000000000..96a540f2001 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DruidPlannerResourceAnalyzeTest.java @@ -0,0 +1,246 @@ +/* + * 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.sql.calcite; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Set; + +public class DruidPlannerResourceAnalyzeTest extends BaseCalciteQueryTest +{ + @Test + public void testTable() + { + final String sql = "SELECT COUNT(*) FROM foo WHERE foo.dim1 <> 'z'"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE) + ), + requiredResources + ); + } + + @Test + public void testConfusingTable() + { + final String sql = "SELECT COUNT(*) FROM foo as druid WHERE druid.dim1 <> 'z'"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE) + ), + requiredResources + ); + } + + @Test + public void testSubquery() + { + final String sql = "SELECT COUNT(*)\n" + + "FROM (\n" + + " SELECT DISTINCT dim2\n" + + " FROM druid.foo\n" + + " WHERE SUBSTRING(dim2, 1, 1) IN (\n" + + " SELECT SUBSTRING(dim1, 1, 1) FROM druid.numfoo WHERE dim1 IS NOT NULL\n" + + " )\n" + + ")"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE), + new Resource("numfoo", ResourceType.DATASOURCE) + ), + requiredResources + ); + } + + @Test + public void testSubqueryUnion() + { + final String sql = "SELECT\n" + + " SUM(cnt),\n" + + " COUNT(*)\n" + + "FROM (\n" + + " SELECT dim2, SUM(cnt) AS cnt\n" + + " FROM (SELECT * FROM druid.foo UNION ALL SELECT * FROM druid.foo2)\n" + + " GROUP BY dim2\n" + + ")"; + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE), + new Resource("foo2", ResourceType.DATASOURCE) + ), + requiredResources + ); + } + + @Test + public void testJoin() + { + final String sql = "SELECT COUNT(*) FROM foo INNER JOIN numfoo ON foo.dim1 = numfoo.dim1 WHERE numfoo.dim1 <> 'z'"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE), + new Resource("numfoo", ResourceType.DATASOURCE) + ), + requiredResources + ); + } + + @Test + public void testView() + { + final String sql = "SELECT COUNT(*) FROM view.aview as druid WHERE dim1_firstchar <> 'z'"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("aview", ResourceType.VIEW) + ), + requiredResources + ); + } + + @Test + public void testSubqueryView() + { + final String sql = "SELECT COUNT(*)\n" + + "FROM (\n" + + " SELECT DISTINCT dim2\n" + + " FROM druid.foo\n" + + " WHERE SUBSTRING(dim2, 1, 1) IN (\n" + + " SELECT SUBSTRING(dim1, 1, 1) FROM view.cview WHERE dim2 IS NOT NULL\n" + + " )\n" + + ")"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE), + new Resource("cview", ResourceType.VIEW) + ), + requiredResources + ); + } + + @Test + public void testJoinView() + { + final String sql = "SELECT COUNT(*) FROM view.cview as aview INNER JOIN numfoo ON aview.dim2 = numfoo.dim2 WHERE numfoo.dim1 <> 'z'"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("cview", ResourceType.VIEW), + new Resource("numfoo", ResourceType.DATASOURCE) + ), + requiredResources + ); + } + + @Test + public void testConfusingViewIdentifiers() + { + final String sql = "SELECT COUNT(*) FROM view.dview as druid WHERE druid.numfoo <> 'z'"; + + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("dview", ResourceType.VIEW) + ), + requiredResources + ); + } + + @Test + public void testDynamicParameters() + { + final String sql = "SELECT SUBSTRING(dim2, CAST(? as BIGINT), CAST(? as BIGINT)) FROM druid.foo LIMIT ?"; + Set requiredResources = analyzeResources( + PLANNER_CONFIG_DEFAULT, + sql, + CalciteTests.REGULAR_USER_AUTH_RESULT + ); + + Assert.assertEquals( + ImmutableSet.of( + new Resource("foo", ResourceType.DATASOURCE) + ), + requiredResources + ); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index b14cec7028d..b97b7f1a489 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -40,8 +40,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; -import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.PlannerConfig; @@ -165,7 +163,6 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe sanityTestVectorizedSqlQueries(PLANNER_FACTORY, query); } - public static void sanityTestVectorizedSqlQueries(PlannerFactory plannerFactory, String query) throws ValidationException, RelConversionException, SqlParseException { @@ -177,12 +174,10 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe QueryContexts.VECTORIZE_KEY, "false", QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, "false" ); - final AuthenticationResult authenticationResult = NoopEscalator.getInstance() - .createEscalatedAuthenticationResult(); try ( - final DruidPlanner vectorPlanner = plannerFactory.createPlanner(vector, ImmutableList.of(), authenticationResult); - final DruidPlanner nonVectorPlanner = plannerFactory.createPlanner(nonvector, ImmutableList.of(), authenticationResult) + final DruidPlanner vectorPlanner = plannerFactory.createPlannerForTesting(vector, query); + final DruidPlanner nonVectorPlanner = plannerFactory.createPlannerForTesting(nonvector, query) ) { final PlannerResult vectorPlan = vectorPlanner.plan(query); final PlannerResult nonVectorPlan = nonVectorPlanner.plan(query); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index 6e05ce8b49c..ebd3ecdf279 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -19,7 +19,6 @@ package org.apache.druid.sql.calcite.expression; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.rel.type.RelDataType; @@ -67,9 +66,7 @@ class ExpressionTestHelper CalciteTests.createOperatorTable(), CalciteTests.createExprMacroTable(), new PlannerConfig(), - ImmutableMap.of(), - ImmutableList.of(), - CalciteTests.REGULAR_USER_AUTH_RESULT + ImmutableMap.of() ); private final RowSignature rowSignature; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java index c9e8a0d7d0f..f225fe62626 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java @@ -157,7 +157,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase { Set sqlSchemas = injector.getInstance(Key.get(new TypeLiteral>(){})); Set> expectedSchemas = - ImmutableSet.of(NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class); + ImmutableSet.of(NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class, NamedViewSchema.class); Assert.assertEquals(expectedSchemas.size(), sqlSchemas.size()); Assert.assertEquals( expectedSchemas, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java index 89f141bc73c..96a464e54d7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java @@ -33,7 +33,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestServerInventoryView; -import org.apache.druid.sql.calcite.view.NoopViewManager; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -58,7 +57,6 @@ public class DruidSchemaNoDataInitTest extends CalciteTestBase new SegmentManager(EasyMock.createMock(SegmentLoader.class)), new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), PLANNER_CONFIG_DEFAULT, - new NoopViewManager(), new NoopEscalator() ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java index 0863cd97fb1..f66bfc00a7d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java @@ -61,7 +61,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestServerInventoryView; -import org.apache.druid.sql.calcite.view.NoopViewManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.SegmentId; @@ -257,7 +256,6 @@ public class DruidSchemaTest extends CalciteTestBase segmentManager, new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)), PLANNER_CONFIG_DEFAULT, - new NoopViewManager(), new NoopEscalator() ) { @@ -276,7 +274,6 @@ public class DruidSchemaTest extends CalciteTestBase segmentManager, new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)), PLANNER_CONFIG_DEFAULT, - new NoopViewManager(), new NoopEscalator() ) { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 4c37c97abb1..61e0a320c56 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -90,7 +90,6 @@ import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestServerInventoryView; -import org.apache.druid.sql.calcite.view.NoopViewManager; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -256,7 +255,6 @@ public class SystemSchemaTest extends CalciteTestBase new SegmentManager(EasyMock.createMock(SegmentLoader.class)), new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), PLANNER_CONFIG_DEFAULT, - new NoopViewManager(), new NoopEscalator() ); druidSchema.start(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index f463c4b3c49..9817908b07d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -118,6 +118,7 @@ import org.apache.druid.sql.calcite.schema.InformationSchema; import org.apache.druid.sql.calcite.schema.LookupSchema; import org.apache.druid.sql.calcite.schema.MetadataSegmentView; import org.apache.druid.sql.calcite.schema.SystemSchema; +import org.apache.druid.sql.calcite.schema.ViewSchema; import org.apache.druid.sql.calcite.view.DruidViewMacroFactory; import org.apache.druid.sql.calcite.view.NoopViewManager; import org.apache.druid.sql.calcite.view.ViewManager; @@ -163,6 +164,7 @@ public class CalciteTests public static final String INFORMATION_SCHEMA_NAME = "INFORMATION_SCHEMA"; public static final String SYSTEM_SCHEMA_NAME = "sys"; public static final String LOOKUP_SCHEMA_NAME = "lookup"; + public static final String VIEW_SCHEMA_NAME = "view"; public static final String TEST_SUPERUSER_NAME = "testSuperuser"; public static final AuthorizerMapper TEST_AUTHORIZER_MAPPER = new AuthorizerMapper(null) @@ -177,6 +179,8 @@ public class CalciteTests if (resource.getType() == ResourceType.DATASOURCE && resource.getName().equals(FORBIDDEN_DATASOURCE)) { return new Access(false); + } else if (resource.getType() == ResourceType.VIEW && resource.getName().equals("forbiddenView")) { + return new Access(false); } else { return Access.OK; } @@ -601,7 +605,8 @@ public class CalciteTests public static final List FORBIDDEN_ROWS = ImmutableList.of( - createRow("2000-01-01", "forbidden", "abcd", 9999.0) + createRow("2000-01-01", "forbidden", "abcd", 9999.0), + createRow("2000-01-02", "forbidden", "a", 1234.0) ); // Hi, I'm Troy McClure. You may remember these rows from such benchmarks generator schemas as basic and expression @@ -1097,6 +1102,7 @@ public class CalciteTests rootSchema.add(CalciteTests.INFORMATION_SCHEMA_NAME, informationSchema); rootSchema.add(CalciteTests.SYSTEM_SCHEMA_NAME, systemSchema); rootSchema.add(CalciteTests.LOOKUP_SCHEMA_NAME, lookupSchema); + rootSchema.add(CalciteTests.VIEW_SCHEMA_NAME, new ViewSchema(viewManager)); return rootSchema; } @@ -1144,7 +1150,6 @@ public class CalciteTests }, createDefaultJoinableFactory(), plannerConfig, - viewManager, TEST_AUTHENTICATOR_ESCALATOR ); 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 722c59fb8ab..a73c759a2e2 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 @@ -794,8 +794,8 @@ public class SqlResourceTest extends CalciteTestBase final QueryException exception = doPost(badQuery).lhs; Assert.assertNotNull(exception); - Assert.assertEquals(exception.getErrorCode(), QueryUnsupportedException.ERROR_CODE); - Assert.assertEquals(exception.getErrorClass(), QueryUnsupportedException.class.getName()); + Assert.assertEquals(QueryUnsupportedException.ERROR_CODE, exception.getErrorCode()); + Assert.assertEquals(QueryUnsupportedException.class.getName(), exception.getErrorClass()); } @Test