mirror of https://github.com/apache/druid.git
Add metrics to the native queries underpinning SQL. (#4561)
* Add metrics to the native queries underpinning SQL. This is done by factoring out the metrics and request log emitting code from QueryResource into a new QueryLifecycle class. That class is used by both QueryResource and the SQL DruidSchema and QueryMaker. Also fixes a couple of bugs in QueryResource: - RequestLogLine start time was set to `TimeUnit.NANOSECONDS.toMillis(startNs)`, which is incorrect since absolute nanos cannot be converted to millis. - DruidMetrics.makeRequestMetrics was called with null `query` on unparseable queries, which led to spurious "Unable to log query" errors. Partial fix for #4047. * Code style * Remove unused imports. * Fix tests. * Remove unused import.
This commit is contained in:
parent
8a4185897e
commit
5048ab3e96
|
@ -38,7 +38,6 @@ import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.groupby.GroupByQuery;
|
import io.druid.query.groupby.GroupByQuery;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.planner.DruidPlanner;
|
import io.druid.sql.calcite.planner.DruidPlanner;
|
||||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||||
import io.druid.sql.calcite.planner.PlannerFactory;
|
import io.druid.sql.calcite.planner.PlannerFactory;
|
||||||
|
@ -116,11 +115,10 @@ public class SqlBenchmark
|
||||||
|
|
||||||
plannerFactory = new PlannerFactory(
|
plannerFactory = new PlannerFactory(
|
||||||
CalciteTests.createMockSchema(walker, plannerConfig),
|
CalciteTests.createMockSchema(walker, plannerConfig),
|
||||||
walker,
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
CalciteTests.createOperatorTable(),
|
CalciteTests.createOperatorTable(),
|
||||||
CalciteTests.createExprMacroTable(),
|
CalciteTests.createExprMacroTable(),
|
||||||
plannerConfig,
|
plannerConfig
|
||||||
new ServerConfig()
|
|
||||||
);
|
);
|
||||||
groupByQuery = GroupByQuery
|
groupByQuery = GroupByQuery
|
||||||
.builder()
|
.builder()
|
||||||
|
|
|
@ -26,7 +26,6 @@ import com.google.common.collect.Iterables;
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.query.Druids;
|
import io.druid.query.Druids;
|
||||||
import io.druid.query.QueryContexts;
|
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.FilteredAggregatorFactory;
|
import io.druid.query.aggregation.FilteredAggregatorFactory;
|
||||||
|
@ -45,7 +44,6 @@ import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import io.druid.segment.virtual.ExpressionVirtualColumn;
|
import io.druid.segment.virtual.ExpressionVirtualColumn;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.filtration.Filtration;
|
import io.druid.sql.calcite.filtration.Filtration;
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||||
|
@ -131,11 +129,10 @@ public class QuantileSqlAggregatorTest
|
||||||
);
|
);
|
||||||
plannerFactory = new PlannerFactory(
|
plannerFactory = new PlannerFactory(
|
||||||
druidSchema,
|
druidSchema,
|
||||||
walker,
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
operatorTable,
|
operatorTable,
|
||||||
CalciteTests.createExprMacroTable(),
|
CalciteTests.createExprMacroTable(),
|
||||||
plannerConfig,
|
plannerConfig
|
||||||
new ServerConfig()
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -223,11 +220,7 @@ public class QuantileSqlAggregatorTest
|
||||||
new QuantilePostAggregator("a7", "a5:agg", 0.999f),
|
new QuantilePostAggregator("a7", "a5:agg", 0.999f),
|
||||||
new QuantilePostAggregator("a8", "a8:agg", 0.50f)
|
new QuantilePostAggregator("a8", "a8:agg", 0.50f)
|
||||||
))
|
))
|
||||||
.context(ImmutableMap.<String, Object>of(
|
.context(ImmutableMap.<String, Object>of("skipEmptyBuckets", true))
|
||||||
"skipEmptyBuckets", true,
|
|
||||||
QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
|
|
||||||
QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
|
|
||||||
))
|
|
||||||
.build(),
|
.build(),
|
||||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||||
);
|
);
|
||||||
|
@ -287,11 +280,7 @@ public class QuantileSqlAggregatorTest
|
||||||
new QuantilePostAggregator("a5", "a5:agg", 0.999f),
|
new QuantilePostAggregator("a5", "a5:agg", 0.999f),
|
||||||
new QuantilePostAggregator("a6", "a4:agg", 0.999f)
|
new QuantilePostAggregator("a6", "a4:agg", 0.999f)
|
||||||
))
|
))
|
||||||
.context(ImmutableMap.<String, Object>of(
|
.context(ImmutableMap.<String, Object>of("skipEmptyBuckets", true))
|
||||||
"skipEmptyBuckets", true,
|
|
||||||
QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
|
|
||||||
QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
|
|
||||||
))
|
|
||||||
.build(),
|
.build(),
|
||||||
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
|
||||||
);
|
);
|
||||||
|
|
|
@ -128,6 +128,15 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes the magical fields added by {@link #makeResponseContextForQuery(Query, long)}.
|
||||||
|
*/
|
||||||
|
public static void removeMagicResponseContextFields(Map<String, Object> responseContext)
|
||||||
|
{
|
||||||
|
responseContext.remove(DirectDruidClient.QUERY_FAIL_TIME);
|
||||||
|
responseContext.remove(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED);
|
||||||
|
}
|
||||||
|
|
||||||
public static Map<String, Object> makeResponseContextForQuery(Query query, long startTimeMillis)
|
public static Map<String, Object> makeResponseContextForQuery(Query query, long startTimeMillis)
|
||||||
{
|
{
|
||||||
final Map<String, Object> responseContext = new MapMaker().makeMap();
|
final Map<String, Object> responseContext = new MapMaker().makeMap();
|
||||||
|
|
|
@ -22,19 +22,13 @@ package io.druid.server;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
|
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
|
||||||
import com.sun.jersey.spi.container.ResourceFilters;
|
import com.sun.jersey.spi.container.ResourceFilters;
|
||||||
import io.druid.client.ServerViewUtil;
|
import io.druid.client.ServerViewUtil;
|
||||||
import io.druid.client.TimelineServerView;
|
import io.druid.client.TimelineServerView;
|
||||||
import io.druid.guice.annotations.Json;
|
import io.druid.guice.annotations.Json;
|
||||||
import io.druid.guice.annotations.Smile;
|
import io.druid.guice.annotations.Smile;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.GenericQueryMetricsFactory;
|
|
||||||
import io.druid.query.QuerySegmentWalker;
|
|
||||||
import io.druid.query.QueryToolChestWarehouse;
|
|
||||||
import io.druid.server.http.security.StateResourceFilter;
|
import io.druid.server.http.security.StateResourceFilter;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.server.log.RequestLogger;
|
|
||||||
import io.druid.server.security.AuthConfig;
|
import io.druid.server.security.AuthConfig;
|
||||||
|
|
||||||
import javax.servlet.http.HttpServletRequest;
|
import javax.servlet.http.HttpServletRequest;
|
||||||
|
@ -59,30 +53,20 @@ public class BrokerQueryResource extends QueryResource
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public BrokerQueryResource(
|
public BrokerQueryResource(
|
||||||
QueryToolChestWarehouse warehouse,
|
QueryLifecycleFactory queryLifecycleFactory,
|
||||||
ServerConfig config,
|
|
||||||
@Json ObjectMapper jsonMapper,
|
@Json ObjectMapper jsonMapper,
|
||||||
@Smile ObjectMapper smileMapper,
|
@Smile ObjectMapper smileMapper,
|
||||||
QuerySegmentWalker texasRanger,
|
|
||||||
ServiceEmitter emitter,
|
|
||||||
RequestLogger requestLogger,
|
|
||||||
QueryManager queryManager,
|
QueryManager queryManager,
|
||||||
AuthConfig authConfig,
|
AuthConfig authConfig,
|
||||||
GenericQueryMetricsFactory queryMetricsFactory,
|
|
||||||
TimelineServerView brokerServerView
|
TimelineServerView brokerServerView
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
warehouse,
|
queryLifecycleFactory,
|
||||||
config,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
smileMapper,
|
smileMapper,
|
||||||
texasRanger,
|
|
||||||
emitter,
|
|
||||||
requestLogger,
|
|
||||||
queryManager,
|
queryManager,
|
||||||
authConfig,
|
authConfig
|
||||||
queryMetricsFactory
|
|
||||||
);
|
);
|
||||||
this.brokerServerView = brokerServerView;
|
this.brokerServerView = brokerServerView;
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,363 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.server;
|
||||||
|
|
||||||
|
import com.google.common.base.Strings;
|
||||||
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import io.druid.client.DirectDruidClient;
|
||||||
|
import io.druid.java.util.common.ISE;
|
||||||
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
|
import io.druid.java.util.common.guava.SequenceWrapper;
|
||||||
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
|
import io.druid.java.util.common.logger.Logger;
|
||||||
|
import io.druid.query.DruidMetrics;
|
||||||
|
import io.druid.query.GenericQueryMetricsFactory;
|
||||||
|
import io.druid.query.Query;
|
||||||
|
import io.druid.query.QueryInterruptedException;
|
||||||
|
import io.druid.query.QueryMetrics;
|
||||||
|
import io.druid.query.QueryPlus;
|
||||||
|
import io.druid.query.QuerySegmentWalker;
|
||||||
|
import io.druid.query.QueryToolChest;
|
||||||
|
import io.druid.query.QueryToolChestWarehouse;
|
||||||
|
import io.druid.server.initialization.ServerConfig;
|
||||||
|
import io.druid.server.log.RequestLogger;
|
||||||
|
import io.druid.server.security.Access;
|
||||||
|
import io.druid.server.security.Action;
|
||||||
|
import io.druid.server.security.AuthConfig;
|
||||||
|
import io.druid.server.security.AuthorizationInfo;
|
||||||
|
import io.druid.server.security.Resource;
|
||||||
|
import io.druid.server.security.ResourceType;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class that helps a Druid server (broker, historical, etc) manage the lifecycle of a query that it is handling. It
|
||||||
|
* ensures that a query goes through the following stages, in the proper order:
|
||||||
|
*
|
||||||
|
* <ol>
|
||||||
|
* <li>Initialization ({@link #initialize(Query)})</li>
|
||||||
|
* <li>Authorization ({@link #authorize(AuthorizationInfo)}</li>
|
||||||
|
* <li>Execution ({@link #execute()}</li>
|
||||||
|
* <li>Logging ({@link #emitLogsAndMetrics(Throwable, String, long)}</li>
|
||||||
|
* </ol>
|
||||||
|
*
|
||||||
|
* This object is not thread-safe.
|
||||||
|
*/
|
||||||
|
public class QueryLifecycle
|
||||||
|
{
|
||||||
|
private static final Logger log = new Logger(QueryLifecycle.class);
|
||||||
|
|
||||||
|
private final QueryToolChestWarehouse warehouse;
|
||||||
|
private final QuerySegmentWalker texasRanger;
|
||||||
|
private final GenericQueryMetricsFactory queryMetricsFactory;
|
||||||
|
private final ServiceEmitter emitter;
|
||||||
|
private final RequestLogger requestLogger;
|
||||||
|
private final ServerConfig serverConfig;
|
||||||
|
private final AuthConfig authConfig;
|
||||||
|
private final long startMs;
|
||||||
|
private final long startNs;
|
||||||
|
|
||||||
|
private State state = State.NEW;
|
||||||
|
private QueryToolChest toolChest;
|
||||||
|
private QueryPlus queryPlus;
|
||||||
|
|
||||||
|
public QueryLifecycle(
|
||||||
|
final QueryToolChestWarehouse warehouse,
|
||||||
|
final QuerySegmentWalker texasRanger,
|
||||||
|
final GenericQueryMetricsFactory queryMetricsFactory,
|
||||||
|
final ServiceEmitter emitter,
|
||||||
|
final RequestLogger requestLogger,
|
||||||
|
final ServerConfig serverConfig,
|
||||||
|
final AuthConfig authConfig,
|
||||||
|
final long startMs,
|
||||||
|
final long startNs
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.warehouse = warehouse;
|
||||||
|
this.texasRanger = texasRanger;
|
||||||
|
this.queryMetricsFactory = queryMetricsFactory;
|
||||||
|
this.emitter = emitter;
|
||||||
|
this.requestLogger = requestLogger;
|
||||||
|
this.serverConfig = serverConfig;
|
||||||
|
this.authConfig = authConfig;
|
||||||
|
this.startMs = startMs;
|
||||||
|
this.startNs = startNs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For callers where simplicity is desired over flexibility. This method does it all in one call. If the request
|
||||||
|
* is unauthorized, an IllegalStateException will be thrown. Logs and metrics are emitted when the Sequence is
|
||||||
|
* either fully iterated or throws an exception.
|
||||||
|
*
|
||||||
|
* @param query the query
|
||||||
|
* @param authorizationInfo authorization info from the request; or null if none is present. This must be non-null
|
||||||
|
* if security is enabled, or the request will be considered unauthorized.
|
||||||
|
* @param remoteAddress remote address, for logging; or null if unknown
|
||||||
|
*
|
||||||
|
* @return results
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public <T> Sequence<T> runSimple(
|
||||||
|
final Query<T> query,
|
||||||
|
@Nullable final AuthorizationInfo authorizationInfo,
|
||||||
|
@Nullable final String remoteAddress
|
||||||
|
)
|
||||||
|
{
|
||||||
|
initialize(query);
|
||||||
|
|
||||||
|
final Sequence<T> results;
|
||||||
|
|
||||||
|
try {
|
||||||
|
final Access access = authorize(authorizationInfo);
|
||||||
|
if (!access.isAllowed()) {
|
||||||
|
throw new ISE("Unauthorized");
|
||||||
|
}
|
||||||
|
|
||||||
|
final QueryLifecycle.QueryResponse queryResponse = execute();
|
||||||
|
results = queryResponse.getResults();
|
||||||
|
}
|
||||||
|
catch (Throwable e) {
|
||||||
|
emitLogsAndMetrics(e, remoteAddress, -1);
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
|
||||||
|
return Sequences.wrap(
|
||||||
|
results,
|
||||||
|
new SequenceWrapper()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void after(final boolean isDone, final Throwable thrown) throws Exception
|
||||||
|
{
|
||||||
|
emitLogsAndMetrics(thrown, remoteAddress, -1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes this object to execute a specific query. Does not actually execute the query.
|
||||||
|
*
|
||||||
|
* @param baseQuery the query
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public void initialize(final Query baseQuery)
|
||||||
|
{
|
||||||
|
transition(State.NEW, State.INITIALIZED);
|
||||||
|
|
||||||
|
String queryId = baseQuery.getId();
|
||||||
|
if (queryId == null) {
|
||||||
|
queryId = UUID.randomUUID().toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
this.queryPlus = QueryPlus.wrap(
|
||||||
|
(Query) DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
|
||||||
|
baseQuery.withId(queryId),
|
||||||
|
serverConfig
|
||||||
|
)
|
||||||
|
);
|
||||||
|
this.toolChest = warehouse.getToolChest(baseQuery);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Authorize the query. Will return an Access object denoting whether the query is authorized or not.
|
||||||
|
*
|
||||||
|
* @param authorizationInfo authorization info from the request; or null if none is present. This must be non-null
|
||||||
|
* if security is enabled, or the request will be considered unauthorized.
|
||||||
|
*
|
||||||
|
* @return authorization result
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if security is enabled and authorizationInfo is null
|
||||||
|
*/
|
||||||
|
public Access authorize(@Nullable final AuthorizationInfo authorizationInfo)
|
||||||
|
{
|
||||||
|
transition(State.INITIALIZED, State.AUTHORIZING);
|
||||||
|
|
||||||
|
if (authConfig.isEnabled()) {
|
||||||
|
// This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424
|
||||||
|
if (authorizationInfo != null) {
|
||||||
|
for (String dataSource : queryPlus.getQuery().getDataSource().getNames()) {
|
||||||
|
Access authResult = authorizationInfo.isAuthorized(
|
||||||
|
new Resource(dataSource, ResourceType.DATASOURCE),
|
||||||
|
Action.READ
|
||||||
|
);
|
||||||
|
if (!authResult.isAllowed()) {
|
||||||
|
// Not authorized; go straight to Jail, do not pass Go.
|
||||||
|
transition(State.AUTHORIZING, State.DONE);
|
||||||
|
return authResult;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
transition(State.AUTHORIZING, State.AUTHORIZED);
|
||||||
|
return new Access(true);
|
||||||
|
} else {
|
||||||
|
throw new ISE("WTF?! Security is enabled but no authorization info found in the request");
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
transition(State.AUTHORIZING, State.AUTHORIZED);
|
||||||
|
return new Access(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execute the query. Can only be called if the query has been authorized. Note that query logs and metrics will
|
||||||
|
* not be emitted automatically when the Sequence is fully iterated. It is the caller's responsibility to call
|
||||||
|
* {@link #emitLogsAndMetrics(Throwable, String, long)} to emit logs and metrics.
|
||||||
|
*
|
||||||
|
* @return result sequence and response context
|
||||||
|
*/
|
||||||
|
public QueryResponse execute()
|
||||||
|
{
|
||||||
|
transition(State.AUTHORIZED, State.EXECUTING);
|
||||||
|
|
||||||
|
final Map<String, Object> responseContext = DirectDruidClient.makeResponseContextForQuery(
|
||||||
|
queryPlus.getQuery(),
|
||||||
|
System.currentTimeMillis()
|
||||||
|
);
|
||||||
|
|
||||||
|
final Sequence res = queryPlus.run(texasRanger, responseContext);
|
||||||
|
|
||||||
|
return new QueryResponse(res == null ? Sequences.empty() : res, responseContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Emit logs and metrics for this query.
|
||||||
|
*
|
||||||
|
* @param e exception that occurred while processing this query
|
||||||
|
* @param remoteAddress remote address, for logging; or null if unknown
|
||||||
|
* @param bytesWritten number of bytes written; will become a query/bytes metric if >= 0
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public void emitLogsAndMetrics(
|
||||||
|
@Nullable final Throwable e,
|
||||||
|
@Nullable final String remoteAddress,
|
||||||
|
final long bytesWritten
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (queryPlus == null) {
|
||||||
|
// Never initialized, don't log or emit anything.
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (state == State.DONE) {
|
||||||
|
log.warn("Tried to emit logs and metrics twice for query[%s]!", queryPlus.getQuery().getId());
|
||||||
|
}
|
||||||
|
|
||||||
|
state = State.DONE;
|
||||||
|
|
||||||
|
final Query query = queryPlus != null ? queryPlus.getQuery() : null;
|
||||||
|
final boolean success = e == null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
final long queryTimeNs = System.nanoTime() - startNs;
|
||||||
|
QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics(
|
||||||
|
queryMetricsFactory,
|
||||||
|
toolChest,
|
||||||
|
queryPlus.getQuery(),
|
||||||
|
Strings.nullToEmpty(remoteAddress)
|
||||||
|
);
|
||||||
|
queryMetrics.success(success);
|
||||||
|
queryMetrics.reportQueryTime(queryTimeNs);
|
||||||
|
|
||||||
|
if (bytesWritten >= 0) {
|
||||||
|
queryMetrics.reportQueryBytes(bytesWritten);
|
||||||
|
}
|
||||||
|
|
||||||
|
queryMetrics.emit(emitter);
|
||||||
|
|
||||||
|
final Map<String, Object> statsMap = new LinkedHashMap<>();
|
||||||
|
statsMap.put("query/time", TimeUnit.NANOSECONDS.toMillis(queryTimeNs));
|
||||||
|
statsMap.put("query/bytes", bytesWritten);
|
||||||
|
statsMap.put("success", success);
|
||||||
|
if (e != null) {
|
||||||
|
statsMap.put("exception", e.toString());
|
||||||
|
|
||||||
|
if (e instanceof QueryInterruptedException) {
|
||||||
|
// Mimic behavior from QueryResource, where this code was originally taken from.
|
||||||
|
log.warn(e, "Exception while processing queryId [%s]", queryPlus.getQuery().getId());
|
||||||
|
statsMap.put("interrupted", true);
|
||||||
|
statsMap.put("reason", e.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
requestLogger.log(
|
||||||
|
new RequestLogLine(
|
||||||
|
new DateTime(startMs),
|
||||||
|
Strings.nullToEmpty(remoteAddress),
|
||||||
|
queryPlus.getQuery(),
|
||||||
|
new QueryStats(statsMap)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
log.error(ex, "Unable to log query [%s]!", query);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Query getQuery()
|
||||||
|
{
|
||||||
|
return queryPlus.getQuery();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void transition(final State from, final State to)
|
||||||
|
{
|
||||||
|
if (state != from) {
|
||||||
|
throw new ISE("Cannot transition from[%s] to[%s].", from, to);
|
||||||
|
}
|
||||||
|
|
||||||
|
state = to;
|
||||||
|
}
|
||||||
|
|
||||||
|
enum State
|
||||||
|
{
|
||||||
|
NEW,
|
||||||
|
INITIALIZED,
|
||||||
|
AUTHORIZING,
|
||||||
|
AUTHORIZED,
|
||||||
|
EXECUTING,
|
||||||
|
DONE
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class QueryResponse
|
||||||
|
{
|
||||||
|
private final Sequence results;
|
||||||
|
private final Map<String, Object> responseContext;
|
||||||
|
|
||||||
|
private QueryResponse(final Sequence results, final Map<String, Object> responseContext)
|
||||||
|
{
|
||||||
|
this.results = results;
|
||||||
|
this.responseContext = responseContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Sequence getResults()
|
||||||
|
{
|
||||||
|
return results;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, Object> getResponseContext()
|
||||||
|
{
|
||||||
|
return responseContext;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,77 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.server;
|
||||||
|
|
||||||
|
import com.google.inject.Inject;
|
||||||
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import io.druid.guice.LazySingleton;
|
||||||
|
import io.druid.query.GenericQueryMetricsFactory;
|
||||||
|
import io.druid.query.QuerySegmentWalker;
|
||||||
|
import io.druid.query.QueryToolChestWarehouse;
|
||||||
|
import io.druid.server.initialization.ServerConfig;
|
||||||
|
import io.druid.server.log.RequestLogger;
|
||||||
|
import io.druid.server.security.AuthConfig;
|
||||||
|
|
||||||
|
@LazySingleton
|
||||||
|
public class QueryLifecycleFactory
|
||||||
|
{
|
||||||
|
private final QueryToolChestWarehouse warehouse;
|
||||||
|
private final QuerySegmentWalker texasRanger;
|
||||||
|
private final GenericQueryMetricsFactory queryMetricsFactory;
|
||||||
|
private final ServiceEmitter emitter;
|
||||||
|
private final RequestLogger requestLogger;
|
||||||
|
private final ServerConfig serverConfig;
|
||||||
|
private final AuthConfig authConfig;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public QueryLifecycleFactory(
|
||||||
|
final QueryToolChestWarehouse warehouse,
|
||||||
|
final QuerySegmentWalker texasRanger,
|
||||||
|
final GenericQueryMetricsFactory queryMetricsFactory,
|
||||||
|
final ServiceEmitter emitter,
|
||||||
|
final RequestLogger requestLogger,
|
||||||
|
final ServerConfig serverConfig,
|
||||||
|
final AuthConfig authConfig
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.warehouse = warehouse;
|
||||||
|
this.texasRanger = texasRanger;
|
||||||
|
this.queryMetricsFactory = queryMetricsFactory;
|
||||||
|
this.emitter = emitter;
|
||||||
|
this.requestLogger = requestLogger;
|
||||||
|
this.serverConfig = serverConfig;
|
||||||
|
this.authConfig = authConfig;
|
||||||
|
}
|
||||||
|
|
||||||
|
public QueryLifecycle factorize()
|
||||||
|
{
|
||||||
|
return new QueryLifecycle(
|
||||||
|
warehouse,
|
||||||
|
texasRanger,
|
||||||
|
queryMetricsFactory,
|
||||||
|
emitter,
|
||||||
|
requestLogger,
|
||||||
|
serverConfig,
|
||||||
|
authConfig,
|
||||||
|
System.currentTimeMillis(),
|
||||||
|
System.nanoTime()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -30,29 +30,17 @@ import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.io.CountingOutputStream;
|
import com.google.common.io.CountingOutputStream;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
|
||||||
import io.druid.client.DirectDruidClient;
|
import io.druid.client.DirectDruidClient;
|
||||||
import io.druid.guice.LazySingleton;
|
import io.druid.guice.LazySingleton;
|
||||||
import io.druid.guice.annotations.Json;
|
import io.druid.guice.annotations.Json;
|
||||||
import io.druid.guice.annotations.Smile;
|
import io.druid.guice.annotations.Smile;
|
||||||
import io.druid.java.util.common.ISE;
|
|
||||||
import io.druid.java.util.common.StringUtils;
|
import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.guava.Sequence;
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
|
||||||
import io.druid.java.util.common.guava.Yielder;
|
import io.druid.java.util.common.guava.Yielder;
|
||||||
import io.druid.java.util.common.guava.Yielders;
|
import io.druid.java.util.common.guava.Yielders;
|
||||||
import io.druid.query.DruidMetrics;
|
|
||||||
import io.druid.query.GenericQueryMetricsFactory;
|
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryContexts;
|
import io.druid.query.QueryContexts;
|
||||||
import io.druid.query.QueryInterruptedException;
|
import io.druid.query.QueryInterruptedException;
|
||||||
import io.druid.query.QueryMetrics;
|
|
||||||
import io.druid.query.QueryPlus;
|
|
||||||
import io.druid.query.QuerySegmentWalker;
|
|
||||||
import io.druid.query.QueryToolChest;
|
|
||||||
import io.druid.query.QueryToolChestWarehouse;
|
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.server.log.RequestLogger;
|
|
||||||
import io.druid.server.metrics.QueryCountStatsProvider;
|
import io.druid.server.metrics.QueryCountStatsProvider;
|
||||||
import io.druid.server.security.Access;
|
import io.druid.server.security.Access;
|
||||||
import io.druid.server.security.Action;
|
import io.druid.server.security.Action;
|
||||||
|
@ -80,8 +68,6 @@ import java.io.InputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -99,48 +85,33 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
public static final String HEADER_IF_NONE_MATCH = "If-None-Match";
|
public static final String HEADER_IF_NONE_MATCH = "If-None-Match";
|
||||||
public static final String HEADER_ETAG = "ETag";
|
public static final String HEADER_ETAG = "ETag";
|
||||||
|
|
||||||
protected final QueryToolChestWarehouse warehouse;
|
protected final QueryLifecycleFactory queryLifecycleFactory;
|
||||||
protected final ServerConfig config;
|
|
||||||
protected final ObjectMapper jsonMapper;
|
protected final ObjectMapper jsonMapper;
|
||||||
protected final ObjectMapper smileMapper;
|
protected final ObjectMapper smileMapper;
|
||||||
protected final ObjectMapper serializeDateTimeAsLongJsonMapper;
|
protected final ObjectMapper serializeDateTimeAsLongJsonMapper;
|
||||||
protected final ObjectMapper serializeDateTimeAsLongSmileMapper;
|
protected final ObjectMapper serializeDateTimeAsLongSmileMapper;
|
||||||
protected final QuerySegmentWalker texasRanger;
|
|
||||||
protected final ServiceEmitter emitter;
|
|
||||||
protected final RequestLogger requestLogger;
|
|
||||||
protected final QueryManager queryManager;
|
protected final QueryManager queryManager;
|
||||||
protected final AuthConfig authConfig;
|
protected final AuthConfig authConfig;
|
||||||
private final GenericQueryMetricsFactory queryMetricsFactory;
|
|
||||||
private final AtomicLong successfulQueryCount = new AtomicLong();
|
private final AtomicLong successfulQueryCount = new AtomicLong();
|
||||||
private final AtomicLong failedQueryCount = new AtomicLong();
|
private final AtomicLong failedQueryCount = new AtomicLong();
|
||||||
private final AtomicLong interruptedQueryCount = new AtomicLong();
|
private final AtomicLong interruptedQueryCount = new AtomicLong();
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public QueryResource(
|
public QueryResource(
|
||||||
QueryToolChestWarehouse warehouse,
|
QueryLifecycleFactory queryLifecycleFactory,
|
||||||
ServerConfig config,
|
|
||||||
@Json ObjectMapper jsonMapper,
|
@Json ObjectMapper jsonMapper,
|
||||||
@Smile ObjectMapper smileMapper,
|
@Smile ObjectMapper smileMapper,
|
||||||
QuerySegmentWalker texasRanger,
|
|
||||||
ServiceEmitter emitter,
|
|
||||||
RequestLogger requestLogger,
|
|
||||||
QueryManager queryManager,
|
QueryManager queryManager,
|
||||||
AuthConfig authConfig,
|
AuthConfig authConfig
|
||||||
GenericQueryMetricsFactory queryMetricsFactory
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.warehouse = warehouse;
|
this.queryLifecycleFactory = queryLifecycleFactory;
|
||||||
this.config = config;
|
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
this.smileMapper = smileMapper;
|
this.smileMapper = smileMapper;
|
||||||
this.serializeDateTimeAsLongJsonMapper = serializeDataTimeAsLong(jsonMapper);
|
this.serializeDateTimeAsLongJsonMapper = serializeDataTimeAsLong(jsonMapper);
|
||||||
this.serializeDateTimeAsLongSmileMapper = serializeDataTimeAsLong(smileMapper);
|
this.serializeDateTimeAsLongSmileMapper = serializeDataTimeAsLong(smileMapper);
|
||||||
this.texasRanger = texasRanger;
|
|
||||||
this.emitter = emitter;
|
|
||||||
this.requestLogger = requestLogger;
|
|
||||||
this.queryManager = queryManager;
|
this.queryManager = queryManager;
|
||||||
this.authConfig = authConfig;
|
this.authConfig = authConfig;
|
||||||
this.queryMetricsFactory = queryMetricsFactory;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@DELETE
|
@DELETE
|
||||||
|
@ -181,35 +152,21 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
@Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE})
|
@Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE})
|
||||||
@Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE, APPLICATION_SMILE})
|
@Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE, APPLICATION_SMILE})
|
||||||
public Response doPost(
|
public Response doPost(
|
||||||
InputStream in,
|
final InputStream in,
|
||||||
@QueryParam("pretty") String pretty,
|
@QueryParam("pretty") final String pretty,
|
||||||
@Context final HttpServletRequest req // used to get request content-type, remote address and AuthorizationInfo
|
@Context final HttpServletRequest req // used to get request content-type, remote address and AuthorizationInfo
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final long startNs = System.nanoTime();
|
final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize();
|
||||||
Query<?> query = null;
|
Query<?> query = null;
|
||||||
QueryToolChest toolChest = null;
|
|
||||||
String queryId = null;
|
|
||||||
|
|
||||||
final ResponseContext context = createContext(req.getContentType(), pretty != null);
|
final ResponseContext context = createContext(req.getContentType(), pretty != null);
|
||||||
|
|
||||||
final String currThreadName = Thread.currentThread().getName();
|
final String currThreadName = Thread.currentThread().getName();
|
||||||
try {
|
try {
|
||||||
|
queryLifecycle.initialize(readQuery(req, in, context));
|
||||||
query = context.getObjectMapper().readValue(in, Query.class);
|
query = queryLifecycle.getQuery();
|
||||||
queryId = query.getId();
|
final String queryId = query.getId();
|
||||||
if (queryId == null) {
|
|
||||||
queryId = UUID.randomUUID().toString();
|
|
||||||
query = query.withId(queryId);
|
|
||||||
}
|
|
||||||
|
|
||||||
query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(query, config);
|
|
||||||
final Map<String, Object> responseContext = DirectDruidClient.makeResponseContextForQuery(
|
|
||||||
query,
|
|
||||||
System.currentTimeMillis()
|
|
||||||
);
|
|
||||||
|
|
||||||
toolChest = warehouse.getToolChest(query);
|
|
||||||
|
|
||||||
Thread.currentThread()
|
Thread.currentThread()
|
||||||
.setName(StringUtils.format("%s[%s_%s_%s]", currThreadName, query.getType(), query.getDataSource().getNames(), queryId));
|
.setName(StringUtils.format("%s[%s_%s_%s]", currThreadName, query.getType(), query.getDataSource().getNames(), queryId));
|
||||||
|
@ -217,49 +174,23 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
log.debug("Got query [%s]", query);
|
log.debug("Got query [%s]", query);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (authConfig.isEnabled()) {
|
final Access authResult = queryLifecycle.authorize((AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN));
|
||||||
// This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424
|
|
||||||
AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN);
|
|
||||||
if (authorizationInfo != null) {
|
|
||||||
for (String dataSource : query.getDataSource().getNames()) {
|
|
||||||
Access authResult = authorizationInfo.isAuthorized(
|
|
||||||
new Resource(dataSource, ResourceType.DATASOURCE),
|
|
||||||
Action.READ
|
|
||||||
);
|
|
||||||
if (!authResult.isAllowed()) {
|
if (!authResult.isAllowed()) {
|
||||||
return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build();
|
return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build();
|
||||||
}
|
}
|
||||||
}
|
|
||||||
} else {
|
|
||||||
throw new ISE("WTF?! Security is enabled but no authorization info found in the request");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
String prevEtag = req.getHeader(HEADER_IF_NONE_MATCH);
|
final QueryLifecycle.QueryResponse queryResponse = queryLifecycle.execute();
|
||||||
if (prevEtag != null) {
|
final Sequence<?> results = queryResponse.getResults();
|
||||||
query = query.withOverriddenContext(
|
final Map<String, Object> responseContext = queryResponse.getResponseContext();
|
||||||
ImmutableMap.of (HEADER_IF_NONE_MATCH, prevEtag)
|
final String prevEtag = getPreviousEtag(req);
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Sequence res = QueryPlus.wrap(query).run(texasRanger, responseContext);
|
|
||||||
|
|
||||||
if (prevEtag != null && prevEtag.equals(responseContext.get(HEADER_ETAG))) {
|
if (prevEtag != null && prevEtag.equals(responseContext.get(HEADER_ETAG))) {
|
||||||
return Response.notModified().build();
|
return Response.notModified().build();
|
||||||
}
|
}
|
||||||
|
|
||||||
final Sequence results;
|
final Yielder<?> yielder = Yielders.each(results);
|
||||||
if (res == null) {
|
|
||||||
results = Sequences.empty();
|
|
||||||
} else {
|
|
||||||
results = res;
|
|
||||||
}
|
|
||||||
|
|
||||||
final Yielder yielder = Yielders.each(results);
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final Query theQuery = query;
|
|
||||||
final QueryToolChest theToolChest = toolChest;
|
|
||||||
boolean shouldFinalize = QueryContexts.isFinalize(query, true);
|
boolean shouldFinalize = QueryContexts.isFinalize(query, true);
|
||||||
boolean serializeDateTimeAsLong =
|
boolean serializeDateTimeAsLong =
|
||||||
QueryContexts.isSerializeDateTimeAsLong(query, false)
|
QueryContexts.isSerializeDateTimeAsLong(query, false)
|
||||||
|
@ -272,8 +203,7 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
@Override
|
@Override
|
||||||
public void write(OutputStream outputStream) throws IOException, WebApplicationException
|
public void write(OutputStream outputStream) throws IOException, WebApplicationException
|
||||||
{
|
{
|
||||||
boolean success = false;
|
Exception e = null;
|
||||||
String exceptionStr = "";
|
|
||||||
|
|
||||||
CountingOutputStream os = new CountingOutputStream(outputStream);
|
CountingOutputStream os = new CountingOutputStream(outputStream);
|
||||||
try {
|
try {
|
||||||
|
@ -282,64 +212,22 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
|
|
||||||
os.flush(); // Some types of OutputStream suppress flush errors in the .close() method.
|
os.flush(); // Some types of OutputStream suppress flush errors in the .close() method.
|
||||||
os.close();
|
os.close();
|
||||||
|
|
||||||
success = true;
|
|
||||||
}
|
}
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
exceptionStr = ex.toString();
|
e = ex;
|
||||||
log.error(ex, "Unable to send query response.");
|
log.error(ex, "Unable to send query response.");
|
||||||
throw Throwables.propagate(ex);
|
throw Throwables.propagate(ex);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
try {
|
Thread.currentThread().setName(currThreadName);
|
||||||
if (success) {
|
|
||||||
|
queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), os.getCount());
|
||||||
|
|
||||||
|
if (e == null) {
|
||||||
successfulQueryCount.incrementAndGet();
|
successfulQueryCount.incrementAndGet();
|
||||||
} else {
|
} else {
|
||||||
failedQueryCount.incrementAndGet();
|
failedQueryCount.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
final long queryTimeNs = System.nanoTime() - startNs;
|
|
||||||
QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics(
|
|
||||||
queryMetricsFactory,
|
|
||||||
theToolChest,
|
|
||||||
theQuery,
|
|
||||||
req.getRemoteAddr()
|
|
||||||
);
|
|
||||||
queryMetrics.success(success);
|
|
||||||
queryMetrics.reportQueryTime(queryTimeNs).emit(emitter);
|
|
||||||
|
|
||||||
DruidMetrics.makeRequestMetrics(
|
|
||||||
queryMetricsFactory,
|
|
||||||
theToolChest,
|
|
||||||
theQuery,
|
|
||||||
req.getRemoteAddr()
|
|
||||||
).reportQueryBytes(os.getCount()).emit(emitter);
|
|
||||||
|
|
||||||
ImmutableMap.Builder<String, Object> statsMapBuilder = ImmutableMap.builder();
|
|
||||||
statsMapBuilder.put("query/time", TimeUnit.NANOSECONDS.toMillis(queryTimeNs));
|
|
||||||
statsMapBuilder.put("query/bytes", os.getCount());
|
|
||||||
statsMapBuilder.put("success", success);
|
|
||||||
if (!success) {
|
|
||||||
statsMapBuilder.put("exception", exceptionStr);
|
|
||||||
}
|
|
||||||
|
|
||||||
requestLogger.log(
|
|
||||||
new RequestLogLine(
|
|
||||||
new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)),
|
|
||||||
req.getRemoteAddr(),
|
|
||||||
theQuery,
|
|
||||||
new QueryStats(
|
|
||||||
statsMapBuilder.build()
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
catch (Exception ex) {
|
|
||||||
log.error(ex, "Unable to log query [%s]!", theQuery);
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
Thread.currentThread().setName(currThreadName);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
@ -352,8 +240,7 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
responseContext.remove(HEADER_ETAG);
|
responseContext.remove(HEADER_ETAG);
|
||||||
}
|
}
|
||||||
|
|
||||||
responseContext.remove(DirectDruidClient.QUERY_FAIL_TIME);
|
DirectDruidClient.removeMagicResponseContextFields(responseContext);
|
||||||
responseContext.remove(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED);
|
|
||||||
|
|
||||||
//Limit the response-context header, see https://github.com/druid-io/druid/issues/2331
|
//Limit the response-context header, see https://github.com/druid-io/druid/issues/2331
|
||||||
//Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString()
|
//Note that Response.ResponseBuilder.header(String key,Object value).build() calls value.toString()
|
||||||
|
@ -379,86 +266,17 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (QueryInterruptedException e) {
|
catch (QueryInterruptedException e) {
|
||||||
try {
|
|
||||||
log.warn(e, "Exception while processing queryId [%s]", queryId);
|
|
||||||
interruptedQueryCount.incrementAndGet();
|
interruptedQueryCount.incrementAndGet();
|
||||||
final long queryTimeNs = System.nanoTime() - startNs;
|
queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1);
|
||||||
QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics(
|
|
||||||
queryMetricsFactory,
|
|
||||||
toolChest,
|
|
||||||
query,
|
|
||||||
req.getRemoteAddr()
|
|
||||||
);
|
|
||||||
queryMetrics.success(false);
|
|
||||||
queryMetrics.reportQueryTime(queryTimeNs).emit(emitter);
|
|
||||||
requestLogger.log(
|
|
||||||
new RequestLogLine(
|
|
||||||
new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)),
|
|
||||||
req.getRemoteAddr(),
|
|
||||||
query,
|
|
||||||
new QueryStats(
|
|
||||||
ImmutableMap.<String, Object>of(
|
|
||||||
"query/time",
|
|
||||||
TimeUnit.NANOSECONDS.toMillis(queryTimeNs),
|
|
||||||
"success",
|
|
||||||
false,
|
|
||||||
"interrupted",
|
|
||||||
true,
|
|
||||||
"reason",
|
|
||||||
e.toString()
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
catch (Exception e2) {
|
|
||||||
log.error(e2, "Unable to log query [%s]!", query);
|
|
||||||
}
|
|
||||||
return context.gotError(e);
|
return context.gotError(e);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
// Input stream has already been consumed by the json object mapper if query == null
|
|
||||||
final String queryString =
|
|
||||||
query == null
|
|
||||||
? "unparsable query"
|
|
||||||
: query.toString();
|
|
||||||
|
|
||||||
log.warn(e, "Exception occurred on request [%s]", queryString);
|
|
||||||
failedQueryCount.incrementAndGet();
|
failedQueryCount.incrementAndGet();
|
||||||
|
queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1);
|
||||||
try {
|
|
||||||
final long queryTimeNs = System.nanoTime() - startNs;
|
|
||||||
QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics(
|
|
||||||
queryMetricsFactory,
|
|
||||||
toolChest,
|
|
||||||
query,
|
|
||||||
req.getRemoteAddr()
|
|
||||||
);
|
|
||||||
queryMetrics.success(false);
|
|
||||||
queryMetrics.reportQueryTime(queryTimeNs).emit(emitter);
|
|
||||||
requestLogger.log(
|
|
||||||
new RequestLogLine(
|
|
||||||
new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)),
|
|
||||||
req.getRemoteAddr(),
|
|
||||||
query,
|
|
||||||
new QueryStats(ImmutableMap.<String, Object>of(
|
|
||||||
"query/time",
|
|
||||||
TimeUnit.NANOSECONDS.toMillis(queryTimeNs),
|
|
||||||
"success",
|
|
||||||
false,
|
|
||||||
"exception",
|
|
||||||
e.toString()
|
|
||||||
))
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
catch (Exception e2) {
|
|
||||||
log.error(e2, "Unable to log query [%s]!", queryString);
|
|
||||||
}
|
|
||||||
|
|
||||||
log.makeAlert(e, "Exception handling request")
|
log.makeAlert(e, "Exception handling request")
|
||||||
.addData("exception", e.toString())
|
.addData("exception", e.toString())
|
||||||
.addData("query", queryString)
|
.addData("query", query != null ? query.toString() : "unparseable query")
|
||||||
.addData("peer", req.getRemoteAddr())
|
.addData("peer", req.getRemoteAddr())
|
||||||
.emit();
|
.emit();
|
||||||
|
|
||||||
|
@ -469,6 +287,29 @@ public class QueryResource implements QueryCountStatsProvider
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static Query<?> readQuery(
|
||||||
|
final HttpServletRequest req,
|
||||||
|
final InputStream in,
|
||||||
|
final ResponseContext context
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
Query baseQuery = context.getObjectMapper().readValue(in, Query.class);
|
||||||
|
String prevEtag = getPreviousEtag(req);
|
||||||
|
|
||||||
|
if (prevEtag != null) {
|
||||||
|
baseQuery = baseQuery.withOverriddenContext(
|
||||||
|
ImmutableMap.of(HEADER_IF_NONE_MATCH, prevEtag)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
return baseQuery;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String getPreviousEtag(final HttpServletRequest req)
|
||||||
|
{
|
||||||
|
return req.getHeader(HEADER_IF_NONE_MATCH);
|
||||||
|
}
|
||||||
|
|
||||||
protected ObjectMapper serializeDataTimeAsLong(ObjectMapper mapper)
|
protected ObjectMapper serializeDataTimeAsLong(ObjectMapper mapper)
|
||||||
{
|
{
|
||||||
return mapper.copy().registerModule(new SimpleModule().addSerializer(DateTime.class, new DateTimeSerializer()));
|
return mapper.copy().registerModule(new SimpleModule().addSerializer(DateTime.class, new DateTimeSerializer()));
|
||||||
|
|
|
@ -0,0 +1,39 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.server.security;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An AuthorizationInfo that is useful for actions generated internally by the system. It allows everything.
|
||||||
|
*/
|
||||||
|
public class SystemAuthorizationInfo implements AuthorizationInfo
|
||||||
|
{
|
||||||
|
public static final SystemAuthorizationInfo INSTANCE = new SystemAuthorizationInfo();
|
||||||
|
|
||||||
|
private SystemAuthorizationInfo()
|
||||||
|
{
|
||||||
|
// Singleton.
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Access isAuthorized(final Resource resource, final Action action)
|
||||||
|
{
|
||||||
|
return new Access(true);
|
||||||
|
}
|
||||||
|
}
|
|
@ -132,16 +132,19 @@ public class QueryResourceTest
|
||||||
EasyMock.expect(testServletRequest.getRemoteAddr()).andReturn("localhost").anyTimes();
|
EasyMock.expect(testServletRequest.getRemoteAddr()).andReturn("localhost").anyTimes();
|
||||||
queryManager = new QueryManager();
|
queryManager = new QueryManager();
|
||||||
queryResource = new QueryResource(
|
queryResource = new QueryResource(
|
||||||
|
new QueryLifecycleFactory(
|
||||||
warehouse,
|
warehouse,
|
||||||
serverConfig,
|
|
||||||
jsonMapper,
|
|
||||||
jsonMapper,
|
|
||||||
testSegmentWalker,
|
testSegmentWalker,
|
||||||
|
new DefaultGenericQueryMetricsFactory(jsonMapper),
|
||||||
new NoopServiceEmitter(),
|
new NoopServiceEmitter(),
|
||||||
new NoopRequestLogger(),
|
new NoopRequestLogger(),
|
||||||
|
serverConfig,
|
||||||
|
new AuthConfig()
|
||||||
|
),
|
||||||
|
jsonMapper,
|
||||||
|
jsonMapper,
|
||||||
queryManager,
|
queryManager,
|
||||||
new AuthConfig(),
|
new AuthConfig()
|
||||||
new DefaultGenericQueryMetricsFactory(jsonMapper)
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -163,6 +166,16 @@ public class QueryResourceTest
|
||||||
@Test
|
@Test
|
||||||
public void testGoodQuery() throws IOException
|
public void testGoodQuery() throws IOException
|
||||||
{
|
{
|
||||||
|
EasyMock.expect(testServletRequest.getAttribute(EasyMock.anyString())).andReturn(
|
||||||
|
new AuthorizationInfo()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Access isAuthorized(Resource resource, Action action)
|
||||||
|
{
|
||||||
|
return new Access(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
).times(1);
|
||||||
EasyMock.replay(testServletRequest);
|
EasyMock.replay(testServletRequest);
|
||||||
Response response = queryResource.doPost(
|
Response response = queryResource.doPost(
|
||||||
new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")),
|
new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")),
|
||||||
|
@ -207,16 +220,19 @@ public class QueryResourceTest
|
||||||
EasyMock.replay(testServletRequest);
|
EasyMock.replay(testServletRequest);
|
||||||
|
|
||||||
queryResource = new QueryResource(
|
queryResource = new QueryResource(
|
||||||
|
new QueryLifecycleFactory(
|
||||||
warehouse,
|
warehouse,
|
||||||
serverConfig,
|
|
||||||
jsonMapper,
|
|
||||||
jsonMapper,
|
|
||||||
testSegmentWalker,
|
testSegmentWalker,
|
||||||
|
new DefaultGenericQueryMetricsFactory(jsonMapper),
|
||||||
new NoopServiceEmitter(),
|
new NoopServiceEmitter(),
|
||||||
new NoopRequestLogger(),
|
new NoopRequestLogger(),
|
||||||
|
serverConfig,
|
||||||
|
new AuthConfig(true)
|
||||||
|
),
|
||||||
|
jsonMapper,
|
||||||
|
jsonMapper,
|
||||||
queryManager,
|
queryManager,
|
||||||
new AuthConfig(true),
|
new AuthConfig(true)
|
||||||
new DefaultGenericQueryMetricsFactory(jsonMapper)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
Response response = queryResource.doPost(
|
Response response = queryResource.doPost(
|
||||||
|
@ -278,16 +294,19 @@ public class QueryResourceTest
|
||||||
EasyMock.replay(testServletRequest);
|
EasyMock.replay(testServletRequest);
|
||||||
|
|
||||||
queryResource = new QueryResource(
|
queryResource = new QueryResource(
|
||||||
|
new QueryLifecycleFactory(
|
||||||
warehouse,
|
warehouse,
|
||||||
serverConfig,
|
|
||||||
jsonMapper,
|
|
||||||
jsonMapper,
|
|
||||||
testSegmentWalker,
|
testSegmentWalker,
|
||||||
|
new DefaultGenericQueryMetricsFactory(jsonMapper),
|
||||||
new NoopServiceEmitter(),
|
new NoopServiceEmitter(),
|
||||||
new NoopRequestLogger(),
|
new NoopRequestLogger(),
|
||||||
|
serverConfig,
|
||||||
|
new AuthConfig(true)
|
||||||
|
),
|
||||||
|
jsonMapper,
|
||||||
|
jsonMapper,
|
||||||
queryManager,
|
queryManager,
|
||||||
new AuthConfig(true),
|
new AuthConfig(true)
|
||||||
new DefaultGenericQueryMetricsFactory(jsonMapper)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\","
|
final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\","
|
||||||
|
@ -375,16 +394,19 @@ public class QueryResourceTest
|
||||||
EasyMock.replay(testServletRequest);
|
EasyMock.replay(testServletRequest);
|
||||||
|
|
||||||
queryResource = new QueryResource(
|
queryResource = new QueryResource(
|
||||||
|
new QueryLifecycleFactory(
|
||||||
warehouse,
|
warehouse,
|
||||||
serverConfig,
|
|
||||||
jsonMapper,
|
|
||||||
jsonMapper,
|
|
||||||
testSegmentWalker,
|
testSegmentWalker,
|
||||||
|
new DefaultGenericQueryMetricsFactory(jsonMapper),
|
||||||
new NoopServiceEmitter(),
|
new NoopServiceEmitter(),
|
||||||
new NoopRequestLogger(),
|
new NoopRequestLogger(),
|
||||||
|
serverConfig,
|
||||||
|
new AuthConfig(true)
|
||||||
|
),
|
||||||
|
jsonMapper,
|
||||||
|
jsonMapper,
|
||||||
queryManager,
|
queryManager,
|
||||||
new AuthConfig(true),
|
new AuthConfig(true)
|
||||||
new DefaultGenericQueryMetricsFactory(jsonMapper)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\","
|
final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\","
|
||||||
|
|
|
@ -21,8 +21,7 @@ package io.druid.sql.calcite.planner;
|
||||||
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import io.druid.math.expr.ExprMacroTable;
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.query.QuerySegmentWalker;
|
import io.druid.server.QueryLifecycleFactory;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.rel.QueryMaker;
|
import io.druid.sql.calcite.rel.QueryMaker;
|
||||||
import io.druid.sql.calcite.schema.DruidSchema;
|
import io.druid.sql.calcite.schema.DruidSchema;
|
||||||
import org.apache.calcite.avatica.util.Casing;
|
import org.apache.calcite.avatica.util.Casing;
|
||||||
|
@ -50,35 +49,32 @@ public class PlannerFactory
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
private final DruidSchema druidSchema;
|
private final DruidSchema druidSchema;
|
||||||
private final QuerySegmentWalker walker;
|
private final QueryLifecycleFactory queryLifecycleFactory;
|
||||||
private final DruidOperatorTable operatorTable;
|
private final DruidOperatorTable operatorTable;
|
||||||
private final ExprMacroTable macroTable;
|
private final ExprMacroTable macroTable;
|
||||||
private final PlannerConfig plannerConfig;
|
private final PlannerConfig plannerConfig;
|
||||||
private final ServerConfig serverConfig;
|
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public PlannerFactory(
|
public PlannerFactory(
|
||||||
final DruidSchema druidSchema,
|
final DruidSchema druidSchema,
|
||||||
final QuerySegmentWalker walker,
|
final QueryLifecycleFactory queryLifecycleFactory,
|
||||||
final DruidOperatorTable operatorTable,
|
final DruidOperatorTable operatorTable,
|
||||||
final ExprMacroTable macroTable,
|
final ExprMacroTable macroTable,
|
||||||
final PlannerConfig plannerConfig,
|
final PlannerConfig plannerConfig
|
||||||
final ServerConfig serverConfig
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.druidSchema = druidSchema;
|
this.druidSchema = druidSchema;
|
||||||
this.walker = walker;
|
this.queryLifecycleFactory = queryLifecycleFactory;
|
||||||
this.operatorTable = operatorTable;
|
this.operatorTable = operatorTable;
|
||||||
this.macroTable = macroTable;
|
this.macroTable = macroTable;
|
||||||
this.plannerConfig = plannerConfig;
|
this.plannerConfig = plannerConfig;
|
||||||
this.serverConfig = serverConfig;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public DruidPlanner createPlanner(final Map<String, Object> queryContext)
|
public DruidPlanner createPlanner(final Map<String, Object> queryContext)
|
||||||
{
|
{
|
||||||
final SchemaPlus rootSchema = Calcites.createRootSchema(druidSchema);
|
final SchemaPlus rootSchema = Calcites.createRootSchema(druidSchema);
|
||||||
final PlannerContext plannerContext = PlannerContext.create(operatorTable, macroTable, plannerConfig, queryContext);
|
final PlannerContext plannerContext = PlannerContext.create(operatorTable, macroTable, plannerConfig, queryContext);
|
||||||
final QueryMaker queryMaker = new QueryMaker(walker, plannerContext, serverConfig);
|
final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext);
|
||||||
final FrameworkConfig frameworkConfig = Frameworks
|
final FrameworkConfig frameworkConfig = Frameworks
|
||||||
.newConfigBuilder()
|
.newConfigBuilder()
|
||||||
.parserConfig(PARSER_CONFIG)
|
.parserConfig(PARSER_CONFIG)
|
||||||
|
|
|
@ -98,7 +98,6 @@ public class DruidNestedGroupBy extends DruidRel<DruidNestedGroupBy>
|
||||||
if (queryDataSource != null) {
|
if (queryDataSource != null) {
|
||||||
return getQueryMaker().runQuery(
|
return getQueryMaker().runQuery(
|
||||||
queryDataSource,
|
queryDataSource,
|
||||||
sourceRel.getOutputRowSignature(),
|
|
||||||
queryBuilder
|
queryBuilder
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -164,7 +164,7 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Object[]> runQuery()
|
public Sequence<Object[]> runQuery()
|
||||||
{
|
{
|
||||||
return getQueryMaker().runQuery(druidTable.getDataSource(), druidTable.getRowSignature(), queryBuilder);
|
return getQueryMaker().runQuery(druidTable.getDataSource(), queryBuilder);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.primitives.Doubles;
|
import com.google.common.primitives.Doubles;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import io.druid.client.DirectDruidClient;
|
|
||||||
import io.druid.common.guava.GuavaUtils;
|
import io.druid.common.guava.GuavaUtils;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.java.util.common.ISE;
|
import io.druid.java.util.common.ISE;
|
||||||
|
@ -33,9 +32,8 @@ import io.druid.java.util.common.guava.Sequence;
|
||||||
import io.druid.java.util.common.guava.Sequences;
|
import io.druid.java.util.common.guava.Sequences;
|
||||||
import io.druid.math.expr.Evals;
|
import io.druid.math.expr.Evals;
|
||||||
import io.druid.query.DataSource;
|
import io.druid.query.DataSource;
|
||||||
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryDataSource;
|
import io.druid.query.QueryDataSource;
|
||||||
import io.druid.query.QueryPlus;
|
|
||||||
import io.druid.query.QuerySegmentWalker;
|
|
||||||
import io.druid.query.Result;
|
import io.druid.query.Result;
|
||||||
import io.druid.query.groupby.GroupByQuery;
|
import io.druid.query.groupby.GroupByQuery;
|
||||||
import io.druid.query.select.EventHolder;
|
import io.druid.query.select.EventHolder;
|
||||||
|
@ -48,10 +46,9 @@ import io.druid.query.topn.DimensionAndMetricValueExtractor;
|
||||||
import io.druid.query.topn.TopNQuery;
|
import io.druid.query.topn.TopNQuery;
|
||||||
import io.druid.query.topn.TopNResultValue;
|
import io.druid.query.topn.TopNResultValue;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
import io.druid.server.QueryLifecycleFactory;
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.PlannerContext;
|
import io.druid.sql.calcite.planner.PlannerContext;
|
||||||
import io.druid.sql.calcite.table.RowSignature;
|
|
||||||
import org.apache.calcite.avatica.ColumnMetaData;
|
import org.apache.calcite.avatica.ColumnMetaData;
|
||||||
import org.apache.calcite.rel.type.RelDataTypeField;
|
import org.apache.calcite.rel.type.RelDataTypeField;
|
||||||
import org.apache.calcite.runtime.Hook;
|
import org.apache.calcite.runtime.Hook;
|
||||||
|
@ -69,19 +66,16 @@ import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
public class QueryMaker
|
public class QueryMaker
|
||||||
{
|
{
|
||||||
private final QuerySegmentWalker walker;
|
private final QueryLifecycleFactory queryLifecycleFactory;
|
||||||
private final PlannerContext plannerContext;
|
private final PlannerContext plannerContext;
|
||||||
private final ServerConfig serverConfig;
|
|
||||||
|
|
||||||
public QueryMaker(
|
public QueryMaker(
|
||||||
final QuerySegmentWalker walker,
|
final QueryLifecycleFactory queryLifecycleFactory,
|
||||||
final PlannerContext plannerContext,
|
final PlannerContext plannerContext
|
||||||
final ServerConfig serverConfig
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.walker = walker;
|
this.queryLifecycleFactory = queryLifecycleFactory;
|
||||||
this.plannerContext = plannerContext;
|
this.plannerContext = plannerContext;
|
||||||
this.serverConfig = serverConfig;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public PlannerContext getPlannerContext()
|
public PlannerContext getPlannerContext()
|
||||||
|
@ -91,7 +85,6 @@ public class QueryMaker
|
||||||
|
|
||||||
public Sequence<Object[]> runQuery(
|
public Sequence<Object[]> runQuery(
|
||||||
final DataSource dataSource,
|
final DataSource dataSource,
|
||||||
final RowSignature sourceRowSignature,
|
|
||||||
final DruidQueryBuilder queryBuilder
|
final DruidQueryBuilder queryBuilder
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -161,31 +154,20 @@ public class QueryMaker
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Object[]> next()
|
public Sequence<Object[]> next()
|
||||||
{
|
{
|
||||||
final SelectQuery queryWithPagination = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
|
final SelectQuery queryWithPagination = baseQuery.withPagingSpec(
|
||||||
baseQuery.withPagingSpec(
|
|
||||||
new PagingSpec(
|
new PagingSpec(
|
||||||
pagingIdentifiers.get(),
|
pagingIdentifiers.get(),
|
||||||
plannerContext.getPlannerConfig().getSelectThreshold(),
|
plannerContext.getPlannerConfig().getSelectThreshold(),
|
||||||
true
|
true
|
||||||
)
|
)
|
||||||
),
|
|
||||||
serverConfig
|
|
||||||
);
|
);
|
||||||
|
|
||||||
Hook.QUERY_PLAN.run(queryWithPagination);
|
|
||||||
|
|
||||||
morePages.set(false);
|
morePages.set(false);
|
||||||
final AtomicBoolean gotResult = new AtomicBoolean();
|
final AtomicBoolean gotResult = new AtomicBoolean();
|
||||||
|
|
||||||
return Sequences.concat(
|
return Sequences.concat(
|
||||||
Sequences.map(
|
Sequences.map(
|
||||||
QueryPlus.wrap(queryWithPagination)
|
runQuery(queryWithPagination),
|
||||||
.run(walker,
|
|
||||||
DirectDruidClient.makeResponseContextForQuery(
|
|
||||||
queryWithPagination,
|
|
||||||
plannerContext.getQueryStartTimeMillis()
|
|
||||||
)
|
|
||||||
),
|
|
||||||
new Function<Result<SelectResultValue>, Sequence<Object[]>>()
|
new Function<Result<SelectResultValue>, Sequence<Object[]>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -244,30 +226,29 @@ public class QueryMaker
|
||||||
return Sequences.concat(sequenceOfSequences);
|
return Sequences.concat(sequenceOfSequences);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private <T> Sequence<T> runQuery(final Query<T> query)
|
||||||
|
{
|
||||||
|
Hook.QUERY_PLAN.run(query);
|
||||||
|
|
||||||
|
// Authorization really should be applied in planning. At this point the query has already begun to execute.
|
||||||
|
// So, use "null" authorizationInfo to force the query to fail if security is enabled.
|
||||||
|
return queryLifecycleFactory.factorize().runSimple(query, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
private Sequence<Object[]> executeTimeseries(
|
private Sequence<Object[]> executeTimeseries(
|
||||||
final DruidQueryBuilder queryBuilder,
|
final DruidQueryBuilder queryBuilder,
|
||||||
final TimeseriesQuery baseQuery
|
final TimeseriesQuery query
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final TimeseriesQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
|
|
||||||
baseQuery,
|
|
||||||
serverConfig
|
|
||||||
);
|
|
||||||
|
|
||||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||||
final String timeOutputName = queryBuilder.getGrouping().getDimensions().isEmpty()
|
final String timeOutputName = queryBuilder.getGrouping().getDimensions().isEmpty()
|
||||||
? null
|
? null
|
||||||
: Iterables.getOnlyElement(queryBuilder.getGrouping().getDimensions())
|
: Iterables.getOnlyElement(queryBuilder.getGrouping().getDimensions())
|
||||||
.getOutputName();
|
.getOutputName();
|
||||||
|
|
||||||
Hook.QUERY_PLAN.run(query);
|
|
||||||
|
|
||||||
return Sequences.map(
|
return Sequences.map(
|
||||||
QueryPlus.wrap(query)
|
runQuery(query),
|
||||||
.run(
|
|
||||||
walker,
|
|
||||||
DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis())
|
|
||||||
),
|
|
||||||
new Function<Result<TimeseriesResultValue>, Object[]>()
|
new Function<Result<TimeseriesResultValue>, Object[]>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -293,25 +274,14 @@ public class QueryMaker
|
||||||
|
|
||||||
private Sequence<Object[]> executeTopN(
|
private Sequence<Object[]> executeTopN(
|
||||||
final DruidQueryBuilder queryBuilder,
|
final DruidQueryBuilder queryBuilder,
|
||||||
final TopNQuery baseQuery
|
final TopNQuery query
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final TopNQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
|
|
||||||
baseQuery,
|
|
||||||
serverConfig
|
|
||||||
);
|
|
||||||
|
|
||||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||||
|
|
||||||
Hook.QUERY_PLAN.run(query);
|
|
||||||
|
|
||||||
return Sequences.concat(
|
return Sequences.concat(
|
||||||
Sequences.map(
|
Sequences.map(
|
||||||
QueryPlus.wrap(query)
|
runQuery(query),
|
||||||
.run(
|
|
||||||
walker,
|
|
||||||
DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis())
|
|
||||||
),
|
|
||||||
new Function<Result<TopNResultValue>, Sequence<Object[]>>()
|
new Function<Result<TopNResultValue>, Sequence<Object[]>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -339,23 +309,13 @@ public class QueryMaker
|
||||||
|
|
||||||
private Sequence<Object[]> executeGroupBy(
|
private Sequence<Object[]> executeGroupBy(
|
||||||
final DruidQueryBuilder queryBuilder,
|
final DruidQueryBuilder queryBuilder,
|
||||||
final GroupByQuery baseQuery
|
final GroupByQuery query
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final GroupByQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
|
|
||||||
baseQuery,
|
|
||||||
serverConfig
|
|
||||||
);
|
|
||||||
|
|
||||||
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
|
||||||
|
|
||||||
Hook.QUERY_PLAN.run(query);
|
|
||||||
return Sequences.map(
|
return Sequences.map(
|
||||||
QueryPlus.wrap(query)
|
runQuery(query),
|
||||||
.run(
|
|
||||||
walker,
|
|
||||||
DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis())
|
|
||||||
),
|
|
||||||
new Function<Row, Object[]>()
|
new Function<Row, Object[]>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -30,7 +30,6 @@ import com.google.common.collect.Sets;
|
||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import io.druid.client.DirectDruidClient;
|
|
||||||
import io.druid.client.ServerView;
|
import io.druid.client.ServerView;
|
||||||
import io.druid.client.TimelineServerView;
|
import io.druid.client.TimelineServerView;
|
||||||
import io.druid.guice.ManageLifecycle;
|
import io.druid.guice.ManageLifecycle;
|
||||||
|
@ -41,8 +40,6 @@ import io.druid.java.util.common.guava.Yielder;
|
||||||
import io.druid.java.util.common.guava.Yielders;
|
import io.druid.java.util.common.guava.Yielders;
|
||||||
import io.druid.java.util.common.lifecycle.LifecycleStart;
|
import io.druid.java.util.common.lifecycle.LifecycleStart;
|
||||||
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
import io.druid.java.util.common.lifecycle.LifecycleStop;
|
||||||
import io.druid.query.QueryPlus;
|
|
||||||
import io.druid.query.QuerySegmentWalker;
|
|
||||||
import io.druid.query.TableDataSource;
|
import io.druid.query.TableDataSource;
|
||||||
import io.druid.query.metadata.metadata.AllColumnIncluderator;
|
import io.druid.query.metadata.metadata.AllColumnIncluderator;
|
||||||
import io.druid.query.metadata.metadata.ColumnAnalysis;
|
import io.druid.query.metadata.metadata.ColumnAnalysis;
|
||||||
|
@ -50,8 +47,9 @@ import io.druid.query.metadata.metadata.SegmentAnalysis;
|
||||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||||
import io.druid.query.spec.MultipleSpecificSegmentSpec;
|
import io.druid.query.spec.MultipleSpecificSegmentSpec;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
|
import io.druid.server.QueryLifecycleFactory;
|
||||||
import io.druid.server.coordination.DruidServerMetadata;
|
import io.druid.server.coordination.DruidServerMetadata;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
import io.druid.server.security.SystemAuthorizationInfo;
|
||||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||||
import io.druid.sql.calcite.table.DruidTable;
|
import io.druid.sql.calcite.table.DruidTable;
|
||||||
import io.druid.sql.calcite.table.RowSignature;
|
import io.druid.sql.calcite.table.RowSignature;
|
||||||
|
@ -91,13 +89,12 @@ public class DruidSchema extends AbstractSchema
|
||||||
private static final EmittingLogger log = new EmittingLogger(DruidSchema.class);
|
private static final EmittingLogger log = new EmittingLogger(DruidSchema.class);
|
||||||
private static final int MAX_SEGMENTS_PER_QUERY = 15000;
|
private static final int MAX_SEGMENTS_PER_QUERY = 15000;
|
||||||
|
|
||||||
private final QuerySegmentWalker walker;
|
private final QueryLifecycleFactory queryLifecycleFactory;
|
||||||
private final TimelineServerView serverView;
|
private final TimelineServerView serverView;
|
||||||
private final PlannerConfig config;
|
private final PlannerConfig config;
|
||||||
private final ViewManager viewManager;
|
private final ViewManager viewManager;
|
||||||
private final ExecutorService cacheExec;
|
private final ExecutorService cacheExec;
|
||||||
private final ConcurrentMap<String, DruidTable> tables;
|
private final ConcurrentMap<String, DruidTable> tables;
|
||||||
private final ServerConfig serverConfig;
|
|
||||||
|
|
||||||
// For awaitInitialization.
|
// For awaitInitialization.
|
||||||
private final CountDownLatch initializationLatch = new CountDownLatch(1);
|
private final CountDownLatch initializationLatch = new CountDownLatch(1);
|
||||||
|
@ -124,20 +121,18 @@ public class DruidSchema extends AbstractSchema
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public DruidSchema(
|
public DruidSchema(
|
||||||
final QuerySegmentWalker walker,
|
final QueryLifecycleFactory queryLifecycleFactory,
|
||||||
final TimelineServerView serverView,
|
final TimelineServerView serverView,
|
||||||
final PlannerConfig config,
|
final PlannerConfig config,
|
||||||
final ViewManager viewManager,
|
final ViewManager viewManager
|
||||||
final ServerConfig serverConfig
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.walker = Preconditions.checkNotNull(walker, "walker");
|
this.queryLifecycleFactory = Preconditions.checkNotNull(queryLifecycleFactory, "queryLifecycleFactory");
|
||||||
this.serverView = Preconditions.checkNotNull(serverView, "serverView");
|
this.serverView = Preconditions.checkNotNull(serverView, "serverView");
|
||||||
this.config = Preconditions.checkNotNull(config, "config");
|
this.config = Preconditions.checkNotNull(config, "config");
|
||||||
this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager");
|
this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager");
|
||||||
this.cacheExec = ScheduledExecutors.fixed(1, "DruidSchema-Cache-%d");
|
this.cacheExec = ScheduledExecutors.fixed(1, "DruidSchema-Cache-%d");
|
||||||
this.tables = Maps.newConcurrentMap();
|
this.tables = Maps.newConcurrentMap();
|
||||||
this.serverConfig = serverConfig;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@LifecycleStart
|
@LifecycleStart
|
||||||
|
@ -404,8 +399,7 @@ public class DruidSchema extends AbstractSchema
|
||||||
|
|
||||||
final Set<DataSegment> retVal = new HashSet<>();
|
final Set<DataSegment> retVal = new HashSet<>();
|
||||||
final Sequence<SegmentAnalysis> sequence = runSegmentMetadataQuery(
|
final Sequence<SegmentAnalysis> sequence = runSegmentMetadataQuery(
|
||||||
walker,
|
queryLifecycleFactory,
|
||||||
serverConfig,
|
|
||||||
Iterables.limit(segments, MAX_SEGMENTS_PER_QUERY)
|
Iterables.limit(segments, MAX_SEGMENTS_PER_QUERY)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -475,8 +469,7 @@ public class DruidSchema extends AbstractSchema
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Sequence<SegmentAnalysis> runSegmentMetadataQuery(
|
private static Sequence<SegmentAnalysis> runSegmentMetadataQuery(
|
||||||
final QuerySegmentWalker walker,
|
final QueryLifecycleFactory queryLifecycleFactory,
|
||||||
final ServerConfig serverConfig,
|
|
||||||
final Iterable<DataSegment> segments
|
final Iterable<DataSegment> segments
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -491,8 +484,7 @@ public class DruidSchema extends AbstractSchema
|
||||||
.map(DataSegment::toDescriptor).collect(Collectors.toList())
|
.map(DataSegment::toDescriptor).collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
|
|
||||||
final SegmentMetadataQuery segmentMetadataQuery = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
|
final SegmentMetadataQuery segmentMetadataQuery = new SegmentMetadataQuery(
|
||||||
new SegmentMetadataQuery(
|
|
||||||
new TableDataSource(dataSource),
|
new TableDataSource(dataSource),
|
||||||
querySegmentSpec,
|
querySegmentSpec,
|
||||||
new AllColumnIncluderator(),
|
new AllColumnIncluderator(),
|
||||||
|
@ -501,18 +493,10 @@ public class DruidSchema extends AbstractSchema
|
||||||
EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class),
|
EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class),
|
||||||
false,
|
false,
|
||||||
false
|
false
|
||||||
),
|
|
||||||
serverConfig
|
|
||||||
);
|
);
|
||||||
|
|
||||||
return QueryPlus.wrap(segmentMetadataQuery)
|
// Use SystemAuthorizationInfo since this is a query generated by Druid itself.
|
||||||
.run(
|
return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, SystemAuthorizationInfo.INSTANCE, null);
|
||||||
walker,
|
|
||||||
DirectDruidClient.makeResponseContextForQuery(
|
|
||||||
segmentMetadataQuery,
|
|
||||||
System.currentTimeMillis()
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis)
|
private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis)
|
||||||
|
|
|
@ -120,7 +120,13 @@ public class DruidAvaticaHandlerTest
|
||||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||||
druidMeta = new DruidMeta(
|
druidMeta = new DruidMeta(
|
||||||
new PlannerFactory(druidSchema, walker, operatorTable, macroTable, plannerConfig, new ServerConfig()),
|
new PlannerFactory(
|
||||||
|
druidSchema,
|
||||||
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
|
operatorTable,
|
||||||
|
macroTable,
|
||||||
|
plannerConfig
|
||||||
|
),
|
||||||
AVATICA_CONFIG
|
AVATICA_CONFIG
|
||||||
);
|
);
|
||||||
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
|
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
|
||||||
|
@ -545,7 +551,13 @@ public class DruidAvaticaHandlerTest
|
||||||
|
|
||||||
final List<Meta.Frame> frames = new ArrayList<>();
|
final List<Meta.Frame> frames = new ArrayList<>();
|
||||||
DruidMeta smallFrameDruidMeta = new DruidMeta(
|
DruidMeta smallFrameDruidMeta = new DruidMeta(
|
||||||
new PlannerFactory(druidSchema, walker, operatorTable, macroTable, plannerConfig, new ServerConfig()),
|
new PlannerFactory(
|
||||||
|
druidSchema,
|
||||||
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
|
operatorTable,
|
||||||
|
macroTable,
|
||||||
|
plannerConfig
|
||||||
|
),
|
||||||
smallFrameConfig
|
smallFrameConfig
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.sql.avatica;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import io.druid.math.expr.ExprMacroTable;
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||||
|
@ -68,11 +67,10 @@ public class DruidStatementTest
|
||||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||||
plannerFactory = new PlannerFactory(
|
plannerFactory = new PlannerFactory(
|
||||||
druidSchema,
|
druidSchema,
|
||||||
walker,
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
operatorTable,
|
operatorTable,
|
||||||
macroTable,
|
macroTable,
|
||||||
plannerConfig,
|
plannerConfig
|
||||||
new ServerConfig()
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -81,7 +81,6 @@ import io.druid.query.topn.TopNQueryBuilder;
|
||||||
import io.druid.segment.column.Column;
|
import io.druid.segment.column.Column;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
import io.druid.segment.virtual.ExpressionVirtualColumn;
|
import io.druid.segment.virtual.ExpressionVirtualColumn;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.filtration.Filtration;
|
import io.druid.sql.calcite.filtration.Filtration;
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||||
|
@ -5354,11 +5353,10 @@ public class CalciteQueryTest
|
||||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||||
druidSchema,
|
druidSchema,
|
||||||
walker,
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
operatorTable,
|
operatorTable,
|
||||||
macroTable,
|
macroTable,
|
||||||
plannerConfig,
|
plannerConfig
|
||||||
new ServerConfig()
|
|
||||||
);
|
);
|
||||||
|
|
||||||
viewManager.createView(
|
viewManager.createView(
|
||||||
|
|
|
@ -29,7 +29,6 @@ import io.druid.java.util.common.Pair;
|
||||||
import io.druid.math.expr.ExprMacroTable;
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
import io.druid.query.QueryInterruptedException;
|
import io.druid.query.QueryInterruptedException;
|
||||||
import io.druid.query.ResourceLimitExceededException;
|
import io.druid.query.ResourceLimitExceededException;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||||
|
@ -80,7 +79,13 @@ public class SqlResourceTest
|
||||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||||
resource = new SqlResource(
|
resource = new SqlResource(
|
||||||
JSON_MAPPER,
|
JSON_MAPPER,
|
||||||
new PlannerFactory(druidSchema, walker, operatorTable, macroTable, plannerConfig, new ServerConfig())
|
new PlannerFactory(
|
||||||
|
druidSchema,
|
||||||
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
|
operatorTable,
|
||||||
|
macroTable,
|
||||||
|
plannerConfig
|
||||||
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,7 +31,6 @@ import io.druid.segment.IndexBuilder;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.segment.TestHelper;
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
import io.druid.sql.calcite.planner.PlannerConfig;
|
import io.druid.sql.calcite.planner.PlannerConfig;
|
||||||
import io.druid.sql.calcite.table.DruidTable;
|
import io.druid.sql.calcite.table.DruidTable;
|
||||||
|
@ -141,11 +140,10 @@ public class DruidSchemaTest
|
||||||
);
|
);
|
||||||
|
|
||||||
schema = new DruidSchema(
|
schema = new DruidSchema(
|
||||||
walker,
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
new TestServerInventoryView(walker.getSegments()),
|
new TestServerInventoryView(walker.getSegments()),
|
||||||
PLANNER_CONFIG_DEFAULT,
|
PLANNER_CONFIG_DEFAULT,
|
||||||
new NoopViewManager(),
|
new NoopViewManager()
|
||||||
new ServerConfig()
|
|
||||||
);
|
);
|
||||||
|
|
||||||
schema.start();
|
schema.start();
|
||||||
|
|
|
@ -30,6 +30,8 @@ import com.google.inject.Guice;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import com.google.inject.Key;
|
import com.google.inject.Key;
|
||||||
import com.google.inject.Module;
|
import com.google.inject.Module;
|
||||||
|
import com.metamx.emitter.core.NoopEmitter;
|
||||||
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import io.druid.collections.StupidPool;
|
import io.druid.collections.StupidPool;
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.data.input.impl.DimensionsSpec;
|
import io.druid.data.input.impl.DimensionsSpec;
|
||||||
|
@ -40,12 +42,16 @@ import io.druid.data.input.impl.TimestampSpec;
|
||||||
import io.druid.guice.ExpressionModule;
|
import io.druid.guice.ExpressionModule;
|
||||||
import io.druid.guice.annotations.Json;
|
import io.druid.guice.annotations.Json;
|
||||||
import io.druid.math.expr.ExprMacroTable;
|
import io.druid.math.expr.ExprMacroTable;
|
||||||
|
import io.druid.query.DefaultGenericQueryMetricsFactory;
|
||||||
import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
|
import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
|
||||||
import io.druid.query.DruidProcessingConfig;
|
import io.druid.query.DruidProcessingConfig;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunnerFactory;
|
import io.druid.query.QueryRunnerFactory;
|
||||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import io.druid.query.QueryRunnerTestHelper;
|
import io.druid.query.QueryRunnerTestHelper;
|
||||||
|
import io.druid.query.QuerySegmentWalker;
|
||||||
|
import io.druid.query.QueryToolChest;
|
||||||
|
import io.druid.query.QueryToolChestWarehouse;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||||
|
@ -77,7 +83,10 @@ import io.druid.segment.IndexBuilder;
|
||||||
import io.druid.segment.QueryableIndex;
|
import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.segment.TestHelper;
|
import io.druid.segment.TestHelper;
|
||||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
|
import io.druid.server.QueryLifecycleFactory;
|
||||||
import io.druid.server.initialization.ServerConfig;
|
import io.druid.server.initialization.ServerConfig;
|
||||||
|
import io.druid.server.log.NoopRequestLogger;
|
||||||
|
import io.druid.server.security.AuthConfig;
|
||||||
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
import io.druid.sql.calcite.aggregation.SqlAggregator;
|
||||||
import io.druid.sql.calcite.expression.SqlOperatorConversion;
|
import io.druid.sql.calcite.expression.SqlOperatorConversion;
|
||||||
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
import io.druid.sql.calcite.planner.DruidOperatorTable;
|
||||||
|
@ -272,6 +281,26 @@ public class CalciteTests
|
||||||
return CONGLOMERATE;
|
return CONGLOMERATE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static QueryLifecycleFactory createMockQueryLifecycleFactory(final QuerySegmentWalker walker)
|
||||||
|
{
|
||||||
|
return new QueryLifecycleFactory(
|
||||||
|
new QueryToolChestWarehouse()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
|
||||||
|
{
|
||||||
|
return CONGLOMERATE.findFactory(query).getToolchest();
|
||||||
|
}
|
||||||
|
},
|
||||||
|
walker,
|
||||||
|
new DefaultGenericQueryMetricsFactory(INJECTOR.getInstance(Key.get(ObjectMapper.class, Json.class))),
|
||||||
|
new ServiceEmitter("dummy", "dummy", new NoopEmitter()),
|
||||||
|
new NoopRequestLogger(),
|
||||||
|
new ServerConfig(),
|
||||||
|
new AuthConfig()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmpDir)
|
public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmpDir)
|
||||||
{
|
{
|
||||||
final QueryableIndex index1 = IndexBuilder.create()
|
final QueryableIndex index1 = IndexBuilder.create()
|
||||||
|
@ -353,11 +382,10 @@ public class CalciteTests
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final DruidSchema schema = new DruidSchema(
|
final DruidSchema schema = new DruidSchema(
|
||||||
walker,
|
CalciteTests.createMockQueryLifecycleFactory(walker),
|
||||||
new TestServerInventoryView(walker.getSegments()),
|
new TestServerInventoryView(walker.getSegments()),
|
||||||
plannerConfig,
|
plannerConfig,
|
||||||
viewManager,
|
viewManager
|
||||||
new ServerConfig()
|
|
||||||
);
|
);
|
||||||
|
|
||||||
schema.start();
|
schema.start();
|
||||||
|
|
Loading…
Reference in New Issue