mirror of https://github.com/apache/druid.git
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:
parent
3785ad5812
commit
fe30f4b414
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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");
|
||||||
|
|
|
@ -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
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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")),
|
||||||
|
|
|
@ -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
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -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()
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
Loading…
Reference in New Issue