refactor sql lifecycle, druid planner, views, and view permissions (#10812)

* before i leaped i should've seen, the view from halfway down

* fixes

* fixes, more test

* rename

* fix style

* further refactoring

* review stuffs

* rename

* more javadoc and comments
This commit is contained in:
Clint Wylie 2021-02-05 12:56:55 -08:00 committed by GitHub
parent 3785ad5812
commit fe30f4b414
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 1531 additions and 362 deletions

View File

@ -35,8 +35,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils; 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.Calcites;
import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerConfig;
@ -439,10 +437,9 @@ public class SqlBenchmark
QueryContexts.VECTORIZE_KEY, vectorize, QueryContexts.VECTORIZE_KEY, vectorize,
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
); );
final AuthenticationResult authenticationResult = NoopEscalator.getInstance() final String sql = QUERIES.get(Integer.parseInt(query));
.createEscalatedAuthenticationResult(); try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(context, sql)) {
try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) { final PlannerResult plannerResult = planner.plan(sql);
final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query)));
final Sequence<Object[]> resultSequence = plannerResult.run(); final Sequence<Object[]> resultSequence = plannerResult.run();
final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow); blackhole.consume(lastRow);
@ -458,10 +455,9 @@ public class SqlBenchmark
QueryContexts.VECTORIZE_KEY, vectorize, QueryContexts.VECTORIZE_KEY, vectorize,
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
); );
final AuthenticationResult authenticationResult = NoopEscalator.getInstance() final String sql = QUERIES.get(Integer.parseInt(query));
.createEscalatedAuthenticationResult(); try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(context, sql)) {
try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) { final PlannerResult plannerResult = planner.plan(sql);
final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query)));
blackhole.consume(plannerResult); blackhole.consume(plannerResult);
} }
} }

View File

@ -36,8 +36,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils; 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.SqlVectorizedExpressionSanityTest;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.DruidPlanner;
@ -290,10 +288,9 @@ public class SqlExpressionBenchmark
QueryContexts.VECTORIZE_KEY, vectorize, QueryContexts.VECTORIZE_KEY, vectorize,
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
); );
final AuthenticationResult authenticationResult = NoopEscalator.getInstance() final String sql = QUERIES.get(Integer.parseInt(query));
.createEscalatedAuthenticationResult(); try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(context, sql)) {
try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) { final PlannerResult plannerResult = planner.plan(sql);
final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query)));
final Sequence<Object[]> resultSequence = plannerResult.run(); final Sequence<Object[]> resultSequence = plannerResult.run();
final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow); blackhole.consume(lastRow);

View File

@ -19,7 +19,6 @@
package org.apache.druid.benchmark.query; package org.apache.druid.benchmark.query;
import com.google.common.collect.ImmutableList;
import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.SchemaPlus;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Intervals; 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.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils; 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.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory; import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -164,9 +161,7 @@ public class SqlVsNativeBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS) @OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryPlanner(Blackhole blackhole) throws Exception public void queryPlanner(Blackhole blackhole) throws Exception
{ {
final AuthenticationResult authResult = NoopEscalator.getInstance() try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(null, sqlQuery)) {
.createEscalatedAuthenticationResult();
try (final DruidPlanner planner = plannerFactory.createPlanner(null, ImmutableList.of(), authResult)) {
final PlannerResult plannerResult = planner.plan(sqlQuery); final PlannerResult plannerResult = planner.plan(sqlQuery);
final Sequence<Object[]> resultSequence = plannerResult.run(); final Sequence<Object[]> resultSequence = plannerResult.run();
final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);

View File

@ -1208,6 +1208,16 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu
Action.WRITE 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( ResourceAction configR = new ResourceAction(
new Resource(".*", ResourceType.CONFIG), new Resource(".*", ResourceType.CONFIG),
Action.READ Action.READ
@ -1228,6 +1238,6 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu
Action.WRITE Action.WRITE
); );
return Lists.newArrayList(datasourceR, datasourceW, configR, configW, stateR, stateW); return Lists.newArrayList(datasourceR, datasourceW, viewR, viewW, configR, configW, stateR, stateW);
} }
} }

View File

@ -112,14 +112,15 @@ public class QueryLifecycle
this.startNs = startNs; this.startNs = startNs;
} }
/** /**
* For callers where simplicity is desired over flexibility. This method does it all in one call. If the request * For callers who have already authorized their query, and where simplicity is desired over flexibility. This method
* is unauthorized, an IllegalStateException will be thrown. Logs and metrics are emitted when the Sequence is * does it all in one call. Logs and metrics are emitted when the Sequence is either fully iterated or throws an
* either fully iterated or throws an exception. * exception.
* *
* @param query the query * @param query the query
* @param authenticationResult authentication result indicating identity of the requester * @param authenticationResult authentication result indicating identity of the requester
* @param remoteAddress remote address, for logging; or null if unknown * @param authorizationResult authorization result of requester
* *
* @return results * @return results
*/ */
@ -127,7 +128,7 @@ public class QueryLifecycle
public <T> Sequence<T> runSimple( public <T> Sequence<T> runSimple(
final Query<T> query, final Query<T> query,
final AuthenticationResult authenticationResult, final AuthenticationResult authenticationResult,
@Nullable final String remoteAddress final Access authorizationResult
) )
{ {
initialize(query); initialize(query);
@ -135,8 +136,8 @@ public class QueryLifecycle
final Sequence<T> results; final Sequence<T> results;
try { try {
final Access access = authorize(authenticationResult); preAuthorized(authenticationResult, authorizationResult);
if (!access.isAllowed()) { if (!authorizationResult.isAllowed()) {
throw new ISE("Unauthorized"); throw new ISE("Unauthorized");
} }
@ -144,7 +145,7 @@ public class QueryLifecycle
results = queryResponse.getResults(); results = queryResponse.getResults();
} }
catch (Throwable e) { catch (Throwable e) {
emitLogsAndMetrics(e, remoteAddress, -1); emitLogsAndMetrics(e, null, -1);
throw e; throw e;
} }
@ -155,7 +156,7 @@ public class QueryLifecycle
@Override @Override
public void after(final boolean isDone, final Throwable thrown) 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); 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. * 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) private Access doAuthorize(final AuthenticationResult authenticationResult, final Access authorizationResult)
{ {
Preconditions.checkNotNull(authenticationResult, "authenticationResult"); Preconditions.checkNotNull(authenticationResult, "authenticationResult");

View File

@ -373,4 +373,12 @@ public class AuthorizationUtils
new Resource(input, ResourceType.DATASOURCE), new Resource(input, ResourceType.DATASOURCE),
Action.WRITE Action.WRITE
); );
/**
* Function for the pattern of generating a {@link ResourceAction} for reading from a given {@link Resource}
*/
public static final Function<Resource, ResourceAction> RESOURCE_READ_RA_GENERATOR = input -> new ResourceAction(
input,
Action.READ
);
} }

View File

@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.StringUtils;
public enum ResourceType public enum ResourceType
{ {
DATASOURCE, DATASOURCE,
VIEW,
CONFIG, CONFIG,
STATE; STATE;

View File

@ -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
);
}
}

View File

@ -20,7 +20,9 @@
package org.apache.druid.sql; package org.apache.druid.sql;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.calcite.avatica.remote.TypedValue; import org.apache.calcite.avatica.remote.TypedValue;
import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.parser.SqlParseException; 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.AuthenticationResult;
import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizationUtils;
import org.apache.druid.server.security.ForbiddenException; 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.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.PlannerFactory; import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.planner.PlannerResult; import org.apache.druid.sql.calcite.planner.PlannerResult;
import org.apache.druid.sql.calcite.planner.PrepareResult; 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.SqlParameter;
import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.sql.http.SqlQuery;
@ -61,6 +65,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeUnit; 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. * 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;
* *
* <ol> * <ol>
* <li>Initialization ({@link #initialize(String, Map)})</li> * <li>Initialization ({@link #initialize(String, Map)})</li>
* <li>Planning ({@link #plan(HttpServletRequest)} or {@link #plan(AuthenticationResult)})</li> * <li>Validation and Authorization ({@link #validateAndAuthorize(HttpServletRequest)} or {@link #validateAndAuthorize(AuthenticationResult)})</li>
* <li>Authorization ({@link #authorize()})</li> * <li>Planning ({@link #plan()})</li>
* <li>Execution ({@link #execute()})</li> * <li>Execution ({@link #execute()})</li>
* <li>Logging ({@link #emitLogsAndMetrics(Throwable, String, long)})</li> * <li>Logging ({@link #emitLogsAndMetrics(Throwable, String, long)})</li>
* </ol> * </ol>
@ -88,16 +93,24 @@ public class SqlLifecycle
private final long startNs; private final long startNs;
private final Object lock = new Object(); private final Object lock = new Object();
@GuardedBy("lock")
private State state = State.NEW; private State state = State.NEW;
// init during intialize // init during intialize
@GuardedBy("lock")
private String sql; private String sql;
@GuardedBy("lock")
private Map<String, Object> queryContext; private Map<String, Object> queryContext;
@GuardedBy("lock")
private List<TypedValue> parameters; private List<TypedValue> parameters;
// init during plan // init during plan
@Nullable private HttpServletRequest req; @GuardedBy("lock")
private PlannerContext plannerContext; private PlannerContext plannerContext;
@GuardedBy("lock")
private ValidationResult validationResult;
@GuardedBy("lock")
private PrepareResult prepareResult; private PrepareResult prepareResult;
@GuardedBy("lock")
private PlannerResult plannerResult; private PlannerResult plannerResult;
public SqlLifecycle( public SqlLifecycle(
@ -116,6 +129,11 @@ public class SqlLifecycle
this.parameters = Collections.emptyList(); 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<String, Object> queryContext) public String initialize(String sql, Map<String, Object> queryContext)
{ {
synchronized (lock) { synchronized (lock) {
@ -126,6 +144,7 @@ public class SqlLifecycle
} }
} }
@GuardedBy("lock")
private Map<String, Object> contextWithSqlId(Map<String, Object> queryContext) private Map<String, Object> contextWithSqlId(Map<String, Object> queryContext)
{ {
Map<String, Object> newContext = new HashMap<>(); Map<String, Object> newContext = new HashMap<>();
@ -136,36 +155,155 @@ public class SqlLifecycle
return newContext; return newContext;
} }
@GuardedBy("lock")
private String sqlQueryId() private String sqlQueryId()
{ {
return (String) this.queryContext.get(PlannerContext.CTX_SQL_QUERY_ID); 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<TypedValue> parameters) public void setParameters(List<TypedValue> parameters)
{ {
synchronized (lock) {
this.parameters = parameters; this.parameters = parameters;
if (this.plannerContext != null) {
this.plannerContext.setParameters(parameters);
}
}
} }
public PrepareResult prepare(AuthenticationResult authenticationResult) /**
throws ValidationException, RelConversionException, SqlParseException * 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) { synchronized (lock) {
try (DruidPlanner planner = plannerFactory.createPlanner(queryContext, parameters, authenticationResult)) { 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 // set planner context for logs/metrics in case something explodes early
this.plannerContext = planner.getPlannerContext(); 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); this.prepareResult = planner.prepare(sql);
return prepareResult; 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);
}
} }
} }
private PlannerContext plan(AuthenticationResult authenticationResult) /**
throws RelConversionException * 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) { synchronized (lock) {
transition(State.INITIALIZED, State.PLANNED); transition(State.AUTHORIZED, State.PLANNED);
try (DruidPlanner planner = plannerFactory.createPlanner(queryContext, parameters, authenticationResult)) { Preconditions.checkNotNull(plannerContext, "Cannot plan, plannerContext is null");
this.plannerContext = planner.getPlannerContext(); try (DruidPlanner planner = plannerFactory.createPlannerWithContext(plannerContext)) {
this.plannerResult = planner.plan(sql); this.plannerResult = planner.plan(sql);
} }
// we can't collapse catch clauses since SqlPlanningException has type-sensitive constructors. // 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 * Execute the fully planned query.
{ *
synchronized (lock) { * If successful, the lifecycle will first transition from {@link State#PLANNED} to {@link State#EXECUTING}.
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;
}
public Sequence<Object[]> execute() public Sequence<Object[]> execute()
{ {
synchronized (lock) { synchronized (lock) {
transition(State.AUTHORIZED, State.EXECUTING); transition(State.PLANNED, State.EXECUTING);
return plannerResult.run(); return plannerResult.run();
} }
} }
@ -274,7 +343,8 @@ public class SqlLifecycle
initialize(sql, queryContext); initialize(sql, queryContext);
try { try {
setParameters(SqlQuery.getParameterList(parameters)); setParameters(SqlQuery.getParameterList(parameters));
planAndAuthorize(authenticationResult); validateAndAuthorize(authenticationResult);
plan();
result = execute(); result = execute();
} }
catch (Throwable e) { 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. * Emit logs and metrics for this query.
* *
@ -326,8 +412,11 @@ public class SqlLifecycle
metricBuilder.setDimension("id", plannerContext.getSqlQueryId()); metricBuilder.setDimension("id", plannerContext.getSqlQueryId());
metricBuilder.setDimension("nativeQueryIds", plannerContext.getNativeQueryIds().toString()); metricBuilder.setDimension("nativeQueryIds", plannerContext.getNativeQueryIds().toString());
} }
if (plannerResult != null) { if (validationResult != null) {
metricBuilder.setDimension("dataSource", plannerResult.datasourceNames().toString()); metricBuilder.setDimension(
"dataSource",
validationResult.getResources().stream().map(Resource::getName).collect(Collectors.toList()).toString()
);
} }
metricBuilder.setDimension("remoteAddress", StringUtils.nullToEmptyNonDruidDataString(remoteAddress)); metricBuilder.setDimension("remoteAddress", StringUtils.nullToEmptyNonDruidDataString(remoteAddress));
metricBuilder.setDimension("success", String.valueOf(success)); metricBuilder.setDimension("success", String.valueOf(success));
@ -370,6 +459,7 @@ public class SqlLifecycle
} }
} }
@GuardedBy("lock")
private void transition(final State from, final State to) private void transition(final State from, final State to)
{ {
if (state != from) { if (state != from) {
@ -383,9 +473,9 @@ public class SqlLifecycle
{ {
NEW, NEW,
INITIALIZED, INITIALIZED,
PLANNED,
AUTHORIZING, AUTHORIZING,
AUTHORIZED, AUTHORIZED,
PLANNED,
EXECUTING, EXECUTING,
UNAUTHORIZED, UNAUTHORIZED,
DONE DONE

View File

@ -161,9 +161,9 @@ public class DruidStatement implements Closeable
try { try {
ensure(State.NEW); ensure(State.NEW);
sqlLifecycle.initialize(query, queryContext); sqlLifecycle.initialize(query, queryContext);
sqlLifecycle.validateAndAuthorize(authenticationResult);
this.authenticationResult = authenticationResult; this.authenticationResult = authenticationResult;
PrepareResult prepareResult = sqlLifecycle.prepare(authenticationResult); PrepareResult prepareResult = sqlLifecycle.prepare();
this.maxRowCount = maxRowCount; this.maxRowCount = maxRowCount;
this.query = query; this.query = query;
List<AvaticaParameter> params = new ArrayList<>(); List<AvaticaParameter> params = new ArrayList<>();
@ -196,7 +196,8 @@ public class DruidStatement implements Closeable
ensure(State.PREPARED); ensure(State.PREPARED);
try { try {
sqlLifecycle.setParameters(parameters); sqlLifecycle.setParameters(parameters);
sqlLifecycle.planAndAuthorize(authenticationResult); sqlLifecycle.validateAndAuthorize(authenticationResult);
sqlLifecycle.plan();
final Sequence<Object[]> baseSequence = yielderOpenCloseExecutor.submit(sqlLifecycle::execute).get(); final Sequence<Object[]> baseSequence = yielderOpenCloseExecutor.submit(sqlLifecycle::execute).get();
// We can't apply limits greater than Integer.MAX_VALUE, ignore them. // We can't apply limits greater than Integer.MAX_VALUE, ignore them.

View File

@ -23,7 +23,6 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.base.Suppliers; import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.DataContext; import org.apache.calcite.DataContext;
import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.config.CalciteConnectionConfig; import org.apache.calcite.config.CalciteConnectionConfig;
@ -73,7 +72,6 @@ import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Properties; import java.util.Properties;
import java.util.Set;
public class DruidPlanner implements Closeable public class DruidPlanner implements Closeable
{ {
@ -92,8 +90,44 @@ public class DruidPlanner implements Closeable
this.plannerContext = plannerContext; 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 public PrepareResult prepare(final String sql) throws SqlParseException, ValidationException, RelConversionException
{ {
reset();
SqlNode parsed = planner.parse(sql); SqlNode parsed = planner.parse(sql);
SqlExplain explain = null; SqlExplain explain = null;
if (parsed.getKind() == SqlKind.EXPLAIN) { if (parsed.getKind() == SqlKind.EXPLAIN) {
@ -104,8 +138,6 @@ public class DruidPlanner implements Closeable
RelRoot root = planner.rel(validated); RelRoot root = planner.rel(validated);
RelDataType rowType = root.validatedRowType; 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(); SqlValidator validator = getValidator();
RelDataType parameterTypes = validator.getParameterRowType(validator.validate(parsed)); RelDataType parameterTypes = validator.getParameterRowType(validator.validate(parsed));
@ -116,9 +148,19 @@ public class DruidPlanner implements Closeable
return new PrepareResult(rowType, parameterTypes); 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; SqlExplain explain = null;
SqlNode parsed = planner.parse(sql); SqlNode parsed = planner.parse(sql);
if (parsed.getKind() == SqlKind.EXPLAIN) { 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 // the planner's type factory is not available until after parsing
this.rexBuilder = new RexBuilder(planner.getTypeFactory()); 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 SqlNode validated = planner.validate(parametized);
final RelRoot root = planner.rel(validated); final RelRoot root = planner.rel(validated);
@ -159,38 +200,30 @@ public class DruidPlanner implements Closeable
planner.close(); 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()); planner.close();
planner.reset();
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()
);
} }
/**
* Construct a {@link PlannerResult} for a {@link RelNode} that is directly translatable to a native Druid query.
*/
private PlannerResult planWithDruidConvention( private PlannerResult planWithDruidConvention(
final SqlExplain explain, final SqlExplain explain,
final RelRoot root final RelRoot root
@ -198,8 +231,7 @@ public class DruidPlanner implements Closeable
{ {
final RelNode possiblyWrappedRootRel = possiblyWrapRootWithOuterLimitFromContext(root); final RelNode possiblyWrappedRootRel = possiblyWrapRootWithOuterLimitFromContext(root);
RelParameterizerShuttle parametizer = new RelParameterizerShuttle(plannerContext); RelNode parametized = rewriteRelDynamicParameters(possiblyWrappedRootRel);
RelNode parametized = possiblyWrappedRootRel.accept(parametizer);
final DruidRel<?> druidRel = (DruidRel<?>) planner.transform( final DruidRel<?> druidRel = (DruidRel<?>) planner.transform(
Rules.DRUID_CONVENTION_RULES, Rules.DRUID_CONVENTION_RULES,
planner.getEmptyTraitSet() planner.getEmptyTraitSet()
@ -208,12 +240,15 @@ public class DruidPlanner implements Closeable
parametized parametized
); );
final Set<String> dataSourceNames = ImmutableSet.copyOf(druidRel.getDataSourceNames());
if (explain != null) { if (explain != null) {
return planExplanation(druidRel, explain, dataSourceNames); return planExplanation(druidRel, explain);
} else { } else {
final Supplier<Sequence<Object[]>> resultsSupplier = () -> { final Supplier<Sequence<Object[]>> resultsSupplier = () -> {
// sanity check
Preconditions.checkState(
plannerContext.getResources().isEmpty() == druidRel.getDataSourceNames().isEmpty(),
"Authorization sanity check failed"
);
if (root.isRefTrivial()) { if (root.isRefTrivial()) {
return druidRel.runQuery(); return druidRel.runQuery();
} else { } 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( private PlannerResult planWithBindableConvention(
final SqlExplain explain, final SqlExplain explain,
final RelRoot root final RelRoot root
@ -242,9 +284,7 @@ public class DruidPlanner implements Closeable
{ {
BindableRel bindableRel = (BindableRel) planner.transform( BindableRel bindableRel = (BindableRel) planner.transform(
Rules.BINDABLE_CONVENTION_RULES, Rules.BINDABLE_CONVENTION_RULES,
planner.getEmptyTraitSet() planner.getEmptyTraitSet().replace(BindableConvention.INSTANCE).plus(root.collation),
.replace(BindableConvention.INSTANCE)
.plus(root.collation),
root.rel root.rel
); );
@ -265,7 +305,7 @@ public class DruidPlanner implements Closeable
} }
if (explain != null) { if (explain != null) {
return planExplanation(bindableRel, explain, ImmutableSet.of()); return planExplanation(bindableRel, explain);
} else { } else {
final BindableRel theRel = bindableRel; final BindableRel theRel = bindableRel;
final DataContext dataContext = plannerContext.createDataContext( final DataContext dataContext = plannerContext.createDataContext(
@ -273,8 +313,8 @@ public class DruidPlanner implements Closeable
plannerContext.getParameters() plannerContext.getParameters()
); );
final Supplier<Sequence<Object[]>> resultsSupplier = () -> { final Supplier<Sequence<Object[]>> resultsSupplier = () -> {
final Enumerable enumerable = theRel.bind(dataContext); final Enumerable<?> enumerable = theRel.bind(dataContext);
final Enumerator enumerator = enumerable.enumerator(); final Enumerator<?> enumerator = enumerable.enumerator();
return Sequences.withBaggage(new BaseSequence<>( return Sequences.withBaggage(new BaseSequence<>(
new BaseSequence.IteratorMaker<Object[], EnumeratorIterator<Object[]>>() new BaseSequence.IteratorMaker<Object[], EnumeratorIterator<Object[]>>()
{ {
@ -298,17 +338,31 @@ public class DruidPlanner implements Closeable
} }
@Override @Override
public void cleanup(EnumeratorIterator iterFromMake) public void cleanup(EnumeratorIterator<Object[]> iterFromMake)
{ {
} }
} }
), enumerator::close); ), 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<Sequence<Object[]>> 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 * 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 * 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<String> datasourceNames
)
{
final String explanation = RelOptUtil.dumpPlan("", rel, explain.getFormat(), explain.getDetailLevel());
final Supplier<Sequence<Object[]>> 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) private static RelDataType getExplainStructType(RelDataTypeFactory typeFactory)
{ {
return typeFactory.createStructType( 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<T> implements Iterator<T> private static class EnumeratorIterator<T> implements Iterator<T>
{ {
private final Iterator<T> it; private final Iterator<T> it;

View File

@ -29,14 +29,18 @@ import org.apache.calcite.linq4j.QueryProvider;
import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.SchemaPlus;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.math.expr.ExprMacroTable; 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.AuthenticationResult;
import org.apache.druid.server.security.Resource;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
@ -63,29 +67,30 @@ public class PlannerContext
private final PlannerConfig plannerConfig; private final PlannerConfig plannerConfig;
private final DateTime localNow; private final DateTime localNow;
private final Map<String, Object> queryContext; private final Map<String, Object> queryContext;
private final List<TypedValue> parameters;
private final AuthenticationResult authenticationResult;
private final String sqlQueryId; private final String sqlQueryId;
private final List<String> nativeQueryIds = new CopyOnWriteArrayList<>(); private final List<String> nativeQueryIds = new CopyOnWriteArrayList<>();
// bindings for dynamic parameters to bind during planning
private List<TypedValue> 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<Resource> resources = Collections.emptySet();
// result of authorizing set of resources against authentication identity
private Access authorizationResult;
private PlannerContext( private PlannerContext(
final DruidOperatorTable operatorTable, final DruidOperatorTable operatorTable,
final ExprMacroTable macroTable, final ExprMacroTable macroTable,
final PlannerConfig plannerConfig, final PlannerConfig plannerConfig,
final DateTime localNow, final DateTime localNow,
final Map<String, Object> queryContext, final Map<String, Object> queryContext
final List<TypedValue> parameters,
final AuthenticationResult authenticationResult
) )
{ {
this.operatorTable = operatorTable; this.operatorTable = operatorTable;
this.macroTable = macroTable; this.macroTable = macroTable;
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig"); this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
this.queryContext = queryContext != null ? new HashMap<>(queryContext) : new HashMap<>(); this.queryContext = queryContext != null ? new HashMap<>(queryContext) : new HashMap<>();
this.parameters = Preconditions.checkNotNull(parameters);
this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.localNow = Preconditions.checkNotNull(localNow, "localNow");
this.authenticationResult = Preconditions.checkNotNull(authenticationResult, "authenticationResult");
String sqlQueryId = (String) this.queryContext.get(CTX_SQL_QUERY_ID); String sqlQueryId = (String) this.queryContext.get(CTX_SQL_QUERY_ID);
// special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle // special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle
@ -99,9 +104,7 @@ public class PlannerContext
final DruidOperatorTable operatorTable, final DruidOperatorTable operatorTable,
final ExprMacroTable macroTable, final ExprMacroTable macroTable,
final PlannerConfig plannerConfig, final PlannerConfig plannerConfig,
final Map<String, Object> queryContext, final Map<String, Object> queryContext
final List<TypedValue> parameters,
final AuthenticationResult authenticationResult
) )
{ {
final DateTime utcNow; final DateTime utcNow;
@ -132,9 +135,7 @@ public class PlannerContext
macroTable, macroTable,
plannerConfig.withOverrides(queryContext), plannerConfig.withOverrides(queryContext),
utcNow.withZone(timeZone), utcNow.withZone(timeZone),
queryContext, queryContext
parameters,
authenticationResult
); );
} }
@ -204,8 +205,7 @@ public class PlannerContext
new DateTime("1970-01-01T00:00:00.000", localNow.getZone()), new DateTime("1970-01-01T00:00:00.000", localNow.getZone()),
localNow localNow
).toDurationMillis(), ).toDurationMillis(),
DataContext.Variable.TIME_ZONE.camelName, localNow.getZone().toTimeZone().clone(), DataContext.Variable.TIME_ZONE.camelName, localNow.getZone().toTimeZone().clone()
DATA_CTX_AUTHENTICATION_RESULT, authenticationResult
); );
private final Map<String, Object> context; private final Map<String, Object> context;
@ -218,6 +218,9 @@ public class PlannerContext
builder.put("?" + i, parameter.value); builder.put("?" + i, parameter.value);
i++; i++;
} }
if (authenticationResult != null) {
builder.put(DATA_CTX_AUTHENTICATION_RESULT, authenticationResult);
}
context = builder.build(); context = builder.build();
} }
@ -248,4 +251,35 @@ public class PlannerContext
return new DruidDataContext(); return new DruidDataContext();
} }
public Access getAuthorizationResult()
{
return authorizationResult;
}
public void setParameters(List<TypedValue> 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<Resource> getResources()
{
return resources;
}
public void setResources(Set<Resource> resources)
{
this.resources = Preconditions.checkNotNull(resources, "resources");
}
} }

View File

@ -20,8 +20,8 @@
package org.apache.druid.sql.calcite.planner; package org.apache.druid.sql.calcite.planner;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Inject; 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.Casing;
import org.apache.calcite.avatica.util.Quoting; import org.apache.calcite.avatica.util.Quoting;
import org.apache.calcite.config.CalciteConnectionConfig; 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.plan.ConventionTraitDef;
import org.apache.calcite.rel.RelCollationTraitDef; import org.apache.calcite.rel.RelCollationTraitDef;
import org.apache.calcite.schema.SchemaPlus; 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.parser.SqlParser;
import org.apache.calcite.sql.validate.SqlConformance; import org.apache.calcite.sql.validate.SqlConformance;
import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.sql2rel.SqlToRelConverter;
import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.FrameworkConfig;
import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.Frameworks;
import org.apache.calcite.tools.ValidationException;
import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Json;
import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.server.QueryLifecycleFactory; 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.AuthorizerMapper;
import org.apache.druid.server.security.NoopEscalator;
import org.apache.druid.sql.calcite.rel.QueryMaker; import org.apache.druid.sql.calcite.rel.QueryMaker;
import org.apache.druid.sql.calcite.schema.DruidSchemaName; import org.apache.druid.sql.calcite.schema.DruidSchemaName;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
@ -89,21 +91,67 @@ public class PlannerFactory
this.druidSchemaName = druidSchemaName; this.druidSchemaName = druidSchemaName;
} }
public DruidPlanner createPlanner( /**
final Map<String, Object> queryContext, * Create a Druid query planner from an initial query context
final List<TypedValue> parameters, */
final AuthenticationResult authenticationResult public DruidPlanner createPlanner(final Map<String, Object> queryContext)
)
{ {
final PlannerContext plannerContext = PlannerContext.create( final PlannerContext plannerContext = PlannerContext.create(
operatorTable, operatorTable,
macroTable, macroTable,
plannerConfig, plannerConfig,
queryContext, queryContext
parameters,
authenticationResult
); );
final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext, jsonMapper); 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<String, Object> 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 final SqlToRelConverter.Config sqlToRelConverterConfig = SqlToRelConverter
.configBuilder() .configBuilder()
.withExpand(false) .withExpand(false)
@ -111,7 +159,7 @@ public class PlannerFactory
.withTrimUnusedFields(false) .withTrimUnusedFields(false)
.withInSubQueryThreshold(Integer.MAX_VALUE) .withInSubQueryThreshold(Integer.MAX_VALUE)
.build(); .build();
final FrameworkConfig frameworkConfig = Frameworks return Frameworks
.newConfigBuilder() .newConfigBuilder()
.parserConfig(PARSER_CONFIG) .parserConfig(PARSER_CONFIG)
.traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE) .traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE)
@ -152,15 +200,5 @@ public class PlannerFactory
} }
}) })
.build(); .build();
return new DruidPlanner(
frameworkConfig,
plannerContext
);
}
public AuthorizerMapper getAuthorizerMapper()
{
return authorizerMapper;
} }
} }

View File

@ -20,31 +20,33 @@
package org.apache.druid.sql.calcite.planner; package org.apache.druid.sql.calcite.planner;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataType;
import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequence;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean; 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 public class PlannerResult
{ {
private final Supplier<Sequence<Object[]>> resultsSupplier; private final Supplier<Sequence<Object[]>> resultsSupplier;
private final RelDataType rowType; private final RelDataType rowType;
private final Set<String> datasourceNames;
private final AtomicBoolean didRun = new AtomicBoolean(); private final AtomicBoolean didRun = new AtomicBoolean();
public PlannerResult( public PlannerResult(
final Supplier<Sequence<Object[]>> resultsSupplier, final Supplier<Sequence<Object[]>> resultsSupplier,
final RelDataType rowType, final RelDataType rowType
final Set<String> datasourceNames
) )
{ {
this.resultsSupplier = resultsSupplier; this.resultsSupplier = resultsSupplier;
this.rowType = rowType; this.rowType = rowType;
this.datasourceNames = ImmutableSet.copyOf(datasourceNames);
} }
/**
* Run the query
*/
public Sequence<Object[]> run() public Sequence<Object[]> run()
{ {
if (!didRun.compareAndSet(false, true)) { if (!didRun.compareAndSet(false, true)) {
@ -58,9 +60,4 @@ public class PlannerResult
{ {
return rowType; return rowType;
} }
public Set<String> datasourceNames()
{
return datasourceNames;
}
} }

View File

@ -21,6 +21,10 @@ package org.apache.druid.sql.calcite.planner;
import org.apache.calcite.rel.type.RelDataType; 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 public class PrepareResult
{ {
private final RelDataType rowType; private final RelDataType rowType;

View File

@ -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<Resource> 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<String> 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<Resource> getResources()
{
return resources;
}
}

View File

@ -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<Resource> resources;
public ValidationResult(
final Set<Resource> resources
)
{
this.resources = ImmutableSet.copyOf(resources);
}
public Set<Resource> getResources()
{
return resources;
}
}

View File

@ -45,6 +45,7 @@ import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycle;
import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.PlannerContext;
@ -142,13 +143,14 @@ public class QueryMaker
query = query.withSqlQueryId(plannerContext.getSqlQueryId()); query = query.withSqlQueryId(plannerContext.getSqlQueryId());
final AuthenticationResult authenticationResult = plannerContext.getAuthenticationResult(); final AuthenticationResult authenticationResult = plannerContext.getAuthenticationResult();
final Access authorizationResult = plannerContext.getAuthorizationResult();
final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize(); final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize();
// After calling "runSimple" the query will start running. We need to do this before reading the toolChest, since // 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 // 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 // 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.) // tight control over which query types we generate in the SQL layer. They all support array-based results.)
final Sequence<T> results = queryLifecycle.runSimple(query, authenticationResult, null); final Sequence<T> results = queryLifecycle.runSimple(query, authenticationResult, authorizationResult);
//noinspection unchecked //noinspection unchecked
final QueryToolChest<T, Query<T>> toolChest = queryLifecycle.getToolChest(); final QueryToolChest<T, Query<T>> toolChest = queryLifecycle.getToolChest();

View File

@ -60,6 +60,7 @@ public class DruidCalciteSchemaModule implements Module
SqlBindings.addSchema(binder, NamedDruidSchema.class); SqlBindings.addSchema(binder, NamedDruidSchema.class);
SqlBindings.addSchema(binder, NamedSystemSchema.class); SqlBindings.addSchema(binder, NamedSystemSchema.class);
SqlBindings.addSchema(binder, NamedLookupSchema.class); SqlBindings.addSchema(binder, NamedLookupSchema.class);
SqlBindings.addSchema(binder, NamedViewSchema.class);
} }
@Provides @Provides

View File

@ -24,11 +24,9 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Predicates; import com.google.common.base.Predicates;
import com.google.common.collect.FluentIterable; import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableMultimap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.errorprone.annotations.concurrent.GuardedBy;
import com.google.inject.Inject; 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.SegmentManager;
import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType; 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.AuthenticationResult;
import org.apache.druid.server.security.Escalator; import org.apache.druid.server.security.Escalator;
import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.table.DruidTable; 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.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
@ -104,7 +101,6 @@ public class DruidSchema extends AbstractSchema
private final QueryLifecycleFactory queryLifecycleFactory; private final QueryLifecycleFactory queryLifecycleFactory;
private final PlannerConfig config; private final PlannerConfig config;
private final SegmentManager segmentManager; private final SegmentManager segmentManager;
private final ViewManager viewManager;
private final JoinableFactory joinableFactory; private final JoinableFactory joinableFactory;
private final ExecutorService cacheExec; private final ExecutorService cacheExec;
private final ConcurrentMap<String, DruidTable> tables; private final ConcurrentMap<String, DruidTable> tables;
@ -152,7 +148,6 @@ public class DruidSchema extends AbstractSchema
final SegmentManager segmentManager, final SegmentManager segmentManager,
final JoinableFactory joinableFactory, final JoinableFactory joinableFactory,
final PlannerConfig config, final PlannerConfig config,
final ViewManager viewManager,
final Escalator escalator final Escalator escalator
) )
{ {
@ -161,7 +156,6 @@ public class DruidSchema extends AbstractSchema
this.segmentManager = segmentManager; this.segmentManager = segmentManager;
this.joinableFactory = joinableFactory; this.joinableFactory = joinableFactory;
this.config = Preconditions.checkNotNull(config, "config"); this.config = Preconditions.checkNotNull(config, "config");
this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager");
this.cacheExec = Execs.singleThreaded("DruidSchema-Cache-%d"); this.cacheExec = Execs.singleThreaded("DruidSchema-Cache-%d");
this.tables = new ConcurrentHashMap<>(); this.tables = new ConcurrentHashMap<>();
this.escalator = escalator; this.escalator = escalator;
@ -351,16 +345,6 @@ public class DruidSchema extends AbstractSchema
return ImmutableMap.copyOf(tables); return ImmutableMap.copyOf(tables);
} }
@Override
protected Multimap<String, org.apache.calcite.schema.Function> getFunctionMultimap()
{
final ImmutableMultimap.Builder<String, org.apache.calcite.schema.Function> builder = ImmutableMultimap.builder();
for (Map.Entry<String, DruidViewMacro> entry : viewManager.getViews().entrySet()) {
builder.put(entry);
}
return builder.build();
}
@VisibleForTesting @VisibleForTesting
void addSegment(final DruidServerMetadata server, final DataSegment segment) void addSegment(final DruidServerMetadata server, final DataSegment segment)
{ {
@ -689,7 +673,7 @@ public class DruidSchema extends AbstractSchema
false false
); );
return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, authenticationResult, null); return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, authenticationResult, Access.OK);
} }
private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis) private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis)

View File

@ -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;
}
}

View File

@ -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<String, Function> getFunctionMultimap()
{
final ImmutableMultimap.Builder<String, Function> builder = ImmutableMultimap.builder();
for (Map.Entry<String, DruidViewMacro> entry : viewManager.getViews().entrySet()) {
builder.put(entry);
}
return builder.build();
}
}

View File

@ -28,7 +28,6 @@ import org.apache.calcite.schema.FunctionParameter;
import org.apache.calcite.schema.TableMacro; import org.apache.calcite.schema.TableMacro;
import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.schema.TranslatableTable;
import org.apache.calcite.schema.impl.ViewTable; 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.server.security.Escalator;
import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerFactory; import org.apache.druid.sql.calcite.planner.PlannerFactory;
@ -61,10 +60,7 @@ public class DruidViewMacro implements TableMacro
public TranslatableTable apply(final List<Object> arguments) public TranslatableTable apply(final List<Object> arguments)
{ {
final RelDataType rowType; final RelDataType rowType;
// Using an escalator here is a hack, but it's currently needed to get the row type. Ideally, some try (final DruidPlanner planner = plannerFactory.createPlanner(null)) {
// 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)) {
rowType = planner.plan(viewSql).rowType(); rowType = planner.plan(viewSql).rowType();
} }

View File

@ -97,8 +97,8 @@ public class SqlResource
Thread.currentThread().setName(StringUtils.format("sql[%s]", sqlQueryId)); Thread.currentThread().setName(StringUtils.format("sql[%s]", sqlQueryId));
lifecycle.setParameters(sqlQuery.getParameterList()); lifecycle.setParameters(sqlQuery.getParameterList());
lifecycle.validateAndAuthorize(req);
final PlannerContext plannerContext = lifecycle.planAndAuthorize(req); final PlannerContext plannerContext = lifecycle.plan();
final DateTimeZone timeZone = plannerContext.getTimeZone(); final DateTimeZone timeZone = plannerContext.getTimeZone();
// Remember which columns are time-typed, so we can emit ISO8601 instead of millis values. // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values.

View File

@ -67,6 +67,8 @@ import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.ForbiddenException; 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.SqlLifecycleFactory;
import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
@ -99,6 +101,7 @@ import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class BaseCalciteQueryTest extends CalciteTestBase public class BaseCalciteQueryTest extends CalciteTestBase
@ -678,39 +681,12 @@ public class BaseCalciteQueryTest extends CalciteTestBase
final ObjectMapper objectMapper final ObjectMapper objectMapper
) throws Exception ) throws Exception
{ {
final InProcessViewManager viewManager = final SqlLifecycleFactory sqlLifecycleFactory = getSqlLifecycleFactory(
new InProcessViewManager(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR, CalciteTests.DRUID_VIEW_MACRO_FACTORY);
SchemaPlus rootSchema = CalciteTests.createMockRootSchema(
conglomerate,
walker,
plannerConfig, plannerConfig,
viewManager,
authorizerMapper
);
final PlannerFactory plannerFactory = new PlannerFactory(
rootSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
operatorTable, operatorTable,
macroTable, macroTable,
plannerConfig,
authorizerMapper, authorizerMapper,
objectMapper, 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'"
); );
return sqlLifecycleFactory.factorize().runSimple(sql, queryContext, parameters, authenticationResult).toList(); return sqlLifecycleFactory.factorize().runSimple(sql, queryContext, parameters, authenticationResult).toList();
@ -754,6 +730,95 @@ public class BaseCalciteQueryTest extends CalciteTestBase
} }
} }
public Set<Resource> 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() protected void cannotVectorize()
{ {
cannotVectorize = true; cannotVectorize = true;

View File

@ -105,6 +105,7 @@ import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.join.JoinType;
import org.apache.druid.server.QueryLifecycle; import org.apache.druid.server.QueryLifecycle;
import org.apache.druid.server.QueryLifecycleFactory; 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;
import org.apache.druid.sql.SqlPlanningException.PlanningError; import org.apache.druid.sql.SqlPlanningException.PlanningError;
import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.DruidExpression;
@ -793,6 +794,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
ImmutableList.of(), ImmutableList.of(),
ImmutableList.of( ImmutableList.of(
new Object[]{"lookup"}, new Object[]{"lookup"},
new Object[]{"view"},
new Object[]{"druid"}, new Object[]{"druid"},
new Object[]{"sys"}, new Object[]{"sys"},
new Object[]{"INFORMATION_SCHEMA"} 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.DATASOURCE3, "TABLE", "NO", "NO"})
.add(new Object[]{"druid", CalciteTests.SOME_DATASOURCE, "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", 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", "COLUMNS", "SYSTEM_TABLE", "NO", "NO"})
.add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"})
.add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "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", "servers", "SYSTEM_TABLE", "NO", "NO"})
.add(new Object[]{"sys", "supervisors", "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[]{"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() .build()
); );
@ -848,8 +854,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE", "NO", "NO"}) .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE", "NO", "NO"})
.add(new Object[]{"druid", CalciteTests.SOME_DATASOURCE, "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", 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", "COLUMNS", "SYSTEM_TABLE", "NO", "NO"})
.add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"})
.add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "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", "servers", "SYSTEM_TABLE", "NO", "NO"})
.add(new Object[]{"sys", "supervisors", "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[]{"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() .build()
); );
} }
@ -920,7 +930,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
testQuery( testQuery(
"SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n"
+ "FROM INFORMATION_SCHEMA.COLUMNS\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(),
ImmutableList.of( ImmutableList.of(
new Object[]{"dim1_firstchar", "VARCHAR", "YES"} 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 @Test
public void testExplainInformationSchemaColumns() throws Exception public void testExplainInformationSchemaColumns() throws Exception
{ {
@ -1026,6 +1052,108 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
9999.0f, 9999.0f,
NullHandling.defaultDoubleValue(), NullHandling.defaultDoubleValue(),
"\"AQAAAQAAAALFBA==\"" "\"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 public void testCountStarOnView() throws Exception
{ {
testQuery( testQuery(
"SELECT COUNT(*) FROM druid.aview WHERE dim1_firstchar <> 'z'", "SELECT COUNT(*) FROM view.aview WHERE dim1_firstchar <> 'z'",
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1) .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 @Test
public void testExplainCountStarOnView() throws Exception public void testExplainCountStarOnView() throws Exception
{ {
@ -5167,7 +5374,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
+ ", signature=[{a0:LONG}])\n"; + ", signature=[{a0:LONG}])\n";
testQuery( 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(),
ImmutableList.of( ImmutableList.of(
new Object[]{explanation} new Object[]{explanation}
@ -8829,7 +9036,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testFilterOnCurrentTimestampOnView() throws Exception public void testFilterOnCurrentTimestampOnView() throws Exception
{ {
testQuery( testQuery(
"SELECT * FROM bview", "SELECT * FROM view.bview",
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1) .dataSource(CalciteTests.DATASOURCE1)
@ -8854,7 +9061,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
testQuery( testQuery(
PLANNER_CONFIG_DEFAULT, PLANNER_CONFIG_DEFAULT,
QUERY_CONTEXT_LOS_ANGELES, QUERY_CONTEXT_LOS_ANGELES,
"SELECT * FROM bview", "SELECT * FROM view.bview",
CalciteTests.REGULAR_USER_AUTH_RESULT, CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of( ImmutableList.of(
Druids.newTimeseriesQueryBuilder() Druids.newTimeseriesQueryBuilder()
@ -13074,11 +13281,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
QueryLifecycleFactory qlf = CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate); QueryLifecycleFactory qlf = CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate);
QueryLifecycle ql = qlf.factorize(); QueryLifecycle ql = qlf.factorize();
Sequence seq = ql.runSimple( Sequence seq = ql.runSimple(query, CalciteTests.SUPER_USER_AUTH_RESULT, Access.OK);
query,
CalciteTests.SUPER_USER_AUTH_RESULT,
null
);
List<Object> results = seq.toList(); List<Object> results = seq.toList();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of(ResultRow.of("def")), ImmutableList.of(ResultRow.of("def")),

View File

@ -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<Resource> 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<Resource> 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<Resource> 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<Resource> 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<Resource> 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<Resource> 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<Resource> 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<Resource> 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<Resource> 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<Resource> requiredResources = analyzeResources(
PLANNER_CONFIG_DEFAULT,
sql,
CalciteTests.REGULAR_USER_AUTH_RESULT
);
Assert.assertEquals(
ImmutableSet.of(
new Resource("foo", ResourceType.DATASOURCE)
),
requiredResources
);
}
}

View File

@ -40,8 +40,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.generator.SegmentGenerator;
import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils; 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.Calcites;
import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerConfig;
@ -165,7 +163,6 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
sanityTestVectorizedSqlQueries(PLANNER_FACTORY, query); sanityTestVectorizedSqlQueries(PLANNER_FACTORY, query);
} }
public static void sanityTestVectorizedSqlQueries(PlannerFactory plannerFactory, String query) public static void sanityTestVectorizedSqlQueries(PlannerFactory plannerFactory, String query)
throws ValidationException, RelConversionException, SqlParseException throws ValidationException, RelConversionException, SqlParseException
{ {
@ -177,12 +174,10 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
QueryContexts.VECTORIZE_KEY, "false", QueryContexts.VECTORIZE_KEY, "false",
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, "false" QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, "false"
); );
final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
.createEscalatedAuthenticationResult();
try ( try (
final DruidPlanner vectorPlanner = plannerFactory.createPlanner(vector, ImmutableList.of(), authenticationResult); final DruidPlanner vectorPlanner = plannerFactory.createPlannerForTesting(vector, query);
final DruidPlanner nonVectorPlanner = plannerFactory.createPlanner(nonvector, ImmutableList.of(), authenticationResult) final DruidPlanner nonVectorPlanner = plannerFactory.createPlannerForTesting(nonvector, query)
) { ) {
final PlannerResult vectorPlan = vectorPlanner.plan(query); final PlannerResult vectorPlan = vectorPlanner.plan(query);
final PlannerResult nonVectorPlan = nonVectorPlanner.plan(query); final PlannerResult nonVectorPlan = nonVectorPlanner.plan(query);

View File

@ -19,7 +19,6 @@
package org.apache.druid.sql.calcite.expression; package org.apache.druid.sql.calcite.expression;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataType;
@ -67,9 +66,7 @@ class ExpressionTestHelper
CalciteTests.createOperatorTable(), CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(), CalciteTests.createExprMacroTable(),
new PlannerConfig(), new PlannerConfig(),
ImmutableMap.of(), ImmutableMap.of()
ImmutableList.of(),
CalciteTests.REGULAR_USER_AUTH_RESULT
); );
private final RowSignature rowSignature; private final RowSignature rowSignature;

View File

@ -157,7 +157,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase
{ {
Set<NamedSchema> sqlSchemas = injector.getInstance(Key.get(new TypeLiteral<Set<NamedSchema>>(){})); Set<NamedSchema> sqlSchemas = injector.getInstance(Key.get(new TypeLiteral<Set<NamedSchema>>(){}));
Set<Class<? extends NamedSchema>> expectedSchemas = Set<Class<? extends NamedSchema>> 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.size(), sqlSchemas.size());
Assert.assertEquals( Assert.assertEquals(
expectedSchemas, expectedSchemas,

View File

@ -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.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestServerInventoryView; import org.apache.druid.sql.calcite.util.TestServerInventoryView;
import org.apache.druid.sql.calcite.view.NoopViewManager;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -58,7 +57,6 @@ public class DruidSchemaNoDataInitTest extends CalciteTestBase
new SegmentManager(EasyMock.createMock(SegmentLoader.class)), new SegmentManager(EasyMock.createMock(SegmentLoader.class)),
new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()),
PLANNER_CONFIG_DEFAULT, PLANNER_CONFIG_DEFAULT,
new NoopViewManager(),
new NoopEscalator() new NoopEscalator()
); );

View File

@ -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.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestServerInventoryView; 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;
import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
@ -257,7 +256,6 @@ public class DruidSchemaTest extends CalciteTestBase
segmentManager, segmentManager,
new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)), new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)),
PLANNER_CONFIG_DEFAULT, PLANNER_CONFIG_DEFAULT,
new NoopViewManager(),
new NoopEscalator() new NoopEscalator()
) )
{ {
@ -276,7 +274,6 @@ public class DruidSchemaTest extends CalciteTestBase
segmentManager, segmentManager,
new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)), new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)),
PLANNER_CONFIG_DEFAULT, PLANNER_CONFIG_DEFAULT,
new NoopViewManager(),
new NoopEscalator() new NoopEscalator()
) )
{ {

View File

@ -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.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.calcite.util.TestServerInventoryView; 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.CompactionState;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
@ -256,7 +255,6 @@ public class SystemSchemaTest extends CalciteTestBase
new SegmentManager(EasyMock.createMock(SegmentLoader.class)), new SegmentManager(EasyMock.createMock(SegmentLoader.class)),
new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()), new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()),
PLANNER_CONFIG_DEFAULT, PLANNER_CONFIG_DEFAULT,
new NoopViewManager(),
new NoopEscalator() new NoopEscalator()
); );
druidSchema.start(); druidSchema.start();

View File

@ -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.LookupSchema;
import org.apache.druid.sql.calcite.schema.MetadataSegmentView; import org.apache.druid.sql.calcite.schema.MetadataSegmentView;
import org.apache.druid.sql.calcite.schema.SystemSchema; 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.DruidViewMacroFactory;
import org.apache.druid.sql.calcite.view.NoopViewManager; import org.apache.druid.sql.calcite.view.NoopViewManager;
import org.apache.druid.sql.calcite.view.ViewManager; 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 INFORMATION_SCHEMA_NAME = "INFORMATION_SCHEMA";
public static final String SYSTEM_SCHEMA_NAME = "sys"; public static final String SYSTEM_SCHEMA_NAME = "sys";
public static final String LOOKUP_SCHEMA_NAME = "lookup"; 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 String TEST_SUPERUSER_NAME = "testSuperuser";
public static final AuthorizerMapper TEST_AUTHORIZER_MAPPER = new AuthorizerMapper(null) 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)) { if (resource.getType() == ResourceType.DATASOURCE && resource.getName().equals(FORBIDDEN_DATASOURCE)) {
return new Access(false); return new Access(false);
} else if (resource.getType() == ResourceType.VIEW && resource.getName().equals("forbiddenView")) {
return new Access(false);
} else { } else {
return Access.OK; return Access.OK;
} }
@ -601,7 +605,8 @@ public class CalciteTests
public static final List<InputRow> FORBIDDEN_ROWS = ImmutableList.of( public static final List<InputRow> 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 // 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.INFORMATION_SCHEMA_NAME, informationSchema);
rootSchema.add(CalciteTests.SYSTEM_SCHEMA_NAME, systemSchema); rootSchema.add(CalciteTests.SYSTEM_SCHEMA_NAME, systemSchema);
rootSchema.add(CalciteTests.LOOKUP_SCHEMA_NAME, lookupSchema); rootSchema.add(CalciteTests.LOOKUP_SCHEMA_NAME, lookupSchema);
rootSchema.add(CalciteTests.VIEW_SCHEMA_NAME, new ViewSchema(viewManager));
return rootSchema; return rootSchema;
} }
@ -1144,7 +1150,6 @@ public class CalciteTests
}, },
createDefaultJoinableFactory(), createDefaultJoinableFactory(),
plannerConfig, plannerConfig,
viewManager,
TEST_AUTHENTICATOR_ESCALATOR TEST_AUTHENTICATOR_ESCALATOR
); );

View File

@ -794,8 +794,8 @@ public class SqlResourceTest extends CalciteTestBase
final QueryException exception = doPost(badQuery).lhs; final QueryException exception = doPost(badQuery).lhs;
Assert.assertNotNull(exception); Assert.assertNotNull(exception);
Assert.assertEquals(exception.getErrorCode(), QueryUnsupportedException.ERROR_CODE); Assert.assertEquals(QueryUnsupportedException.ERROR_CODE, exception.getErrorCode());
Assert.assertEquals(exception.getErrorClass(), QueryUnsupportedException.class.getName()); Assert.assertEquals(QueryUnsupportedException.class.getName(), exception.getErrorClass());
} }
@Test @Test