Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import io.druid.segment.TestHelper;
import io.druid.segment.column.ValueType;
import io.druid.segment.serde.ComplexMetrics;
import io.druid.server.initialization.ServerConfig;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.DruidPlanner;
import io.druid.sql.calcite.planner.PlannerConfig;
Expand Down Expand Up @@ -177,7 +178,8 @@ protected Map<String, Table> getTableMap()
Calcites.createRootSchema(druidSchema),
walker,
CalciteTests.createOperatorTable(),
plannerConfig
plannerConfig,
new ServerConfig()
);
groupByQuery = GroupByQuery
.builder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryContexts;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
Expand All @@ -42,6 +43,7 @@
import io.druid.segment.QueryableIndex;
import io.druid.segment.TestHelper;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.server.initialization.ServerConfig;
import io.druid.sql.calcite.aggregation.SqlAggregator;
import io.druid.sql.calcite.expression.SqlExtractionOperator;
import io.druid.sql.calcite.filtration.Filtration;
Expand Down Expand Up @@ -134,7 +136,7 @@ public void setUp() throws Exception
ImmutableSet.<SqlAggregator>of(new QuantileSqlAggregator()),
ImmutableSet.<SqlExtractionOperator>of()
);
plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig);
plannerFactory = new PlannerFactory(rootSchema, walker, operatorTable, plannerConfig, new ServerConfig());
}

@After
Expand Down Expand Up @@ -200,7 +202,11 @@ public void testQuantileOnFloatAndLongs() throws Exception
new QuantilePostAggregator("a6", "a4:agg", 0.999f),
new QuantilePostAggregator("a7", "a7:agg", 0.50f)
))
.context(ImmutableMap.<String, Object>of("skipEmptyBuckets", true))
.context(ImmutableMap.<String, Object>of(
"skipEmptyBuckets", true,
QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
Expand Down Expand Up @@ -260,7 +266,11 @@ public void testQuantileOnComplexColumn() throws Exception
new QuantilePostAggregator("a5", "a5:agg", 0.999f),
new QuantilePostAggregator("a6", "a4:agg", 0.999f)
))
.context(ImmutableMap.<String, Object>of("skipEmptyBuckets", true))
.context(ImmutableMap.<String, Object>of(
"skipEmptyBuckets", true,
QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS,
QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE
))
.build(),
Iterables.getOnlyElement(queryLogHook.getRecordedQueries())
);
Expand Down
27 changes: 27 additions & 0 deletions server/src/main/java/io/druid/client/DirectDruidClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.collect.MapMaker;
import com.google.common.collect.Maps;
import com.google.common.io.ByteSource;
import com.google.common.util.concurrent.FutureCallback;
Expand Down Expand Up @@ -64,6 +65,7 @@
import io.druid.query.ResourceLimitExceededException;
import io.druid.query.Result;
import io.druid.query.aggregation.MetricManipulatorFns;
import io.druid.server.initialization.ServerConfig;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBufferInputStream;
import org.jboss.netty.handler.codec.http.HttpChunk;
Expand Down Expand Up @@ -114,6 +116,31 @@ public class DirectDruidClient<T> implements QueryRunner<T>
private final AtomicInteger openConnections;
private final boolean isSmile;

public static <T, QueryType extends Query<T>> QueryType withDefaultTimeoutAndMaxScatterGatherBytes(final QueryType query, ServerConfig serverConfig)
{
return (QueryType) QueryContexts.withMaxScatterGatherBytes(
QueryContexts.withDefaultTimeout(
(Query) query,
serverConfig.getDefaultQueryTimeout()
),
serverConfig.getMaxScatterGatherBytes()
);
}

public static Map<String, Object> makeResponseContextForQuery(Query query, long startTimeMillis)
{
final Map<String, Object> responseContext = new MapMaker().makeMap();
responseContext.put(
DirectDruidClient.QUERY_FAIL_TIME,
startTimeMillis + QueryContexts.getTimeout(query)
);
responseContext.put(
DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED,
new AtomicLong()
);
return responseContext;
}

public DirectDruidClient(
QueryToolChestWarehouse warehouse,
QueryWatcher queryWatcher,
Expand Down
13 changes: 4 additions & 9 deletions server/src/main/java/io/druid/server/QueryResource.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.MapMaker;
import com.google.common.io.CountingOutputStream;
import com.google.inject.Inject;
import com.metamx.emitter.EmittingLogger;
Expand All @@ -41,7 +40,6 @@
import io.druid.query.DruidMetrics;
import io.druid.query.GenericQueryMetricsFactory;
import io.druid.query.Query;
import io.druid.query.QueryContexts;
import io.druid.query.QueryInterruptedException;
import io.druid.query.QueryMetrics;
import io.druid.query.QueryPlus;
Expand Down Expand Up @@ -187,22 +185,19 @@ public Response doPost(

final String currThreadName = Thread.currentThread().getName();
try {
final Map<String, Object> responseContext = new MapMaker().makeMap();

query = context.getObjectMapper().readValue(in, Query.class);
queryId = query.getId();
if (queryId == null) {
queryId = UUID.randomUUID().toString();
query = query.withId(queryId);
}
query = QueryContexts.withDefaultTimeout(query, config.getDefaultQueryTimeout());
query = QueryContexts.withMaxScatterGatherBytes(query, config.getMaxScatterGatherBytes());

responseContext.put(
DirectDruidClient.QUERY_FAIL_TIME,
System.currentTimeMillis() + QueryContexts.getTimeout(query)
query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(query, config);
final Map<String, Object> responseContext = DirectDruidClient.makeResponseContextForQuery(
query,
System.currentTimeMillis()
);
responseContext.put(DirectDruidClient.QUERY_TOTAL_BYTES_GATHERED, new AtomicLong());

toolChest = warehouse.getToolChest(query);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ public class PlannerContext

private final PlannerConfig plannerConfig;
private final DateTime localNow;
private final long queryStartTimeMillis;
private final Map<String, Object> queryContext;

private PlannerContext(
Expand All @@ -53,6 +54,7 @@ private PlannerContext(
this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig");
this.queryContext = queryContext != null ? ImmutableMap.copyOf(queryContext) : ImmutableMap.<String, Object>of();
this.localNow = Preconditions.checkNotNull(localNow, "localNow");
this.queryStartTimeMillis = System.currentTimeMillis();
}

public static PlannerContext create(
Expand Down Expand Up @@ -106,6 +108,11 @@ public Map<String, Object> getQueryContext()
return queryContext;
}

public long getQueryStartTimeMillis()
{
return queryStartTimeMillis;
}

public DataContext createDataContext(final JavaTypeFactory typeFactory)
{
class DruidDataContext implements DataContext
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.google.inject.Inject;
import io.druid.query.QuerySegmentWalker;
import io.druid.server.initialization.ServerConfig;
import io.druid.sql.calcite.rel.QueryMaker;
import io.druid.sql.calcite.schema.DruidSchema;
import org.apache.calcite.avatica.util.Casing;
Expand All @@ -44,25 +45,28 @@ public class PlannerFactory
private final QuerySegmentWalker walker;
private final DruidOperatorTable operatorTable;
private final PlannerConfig plannerConfig;
private final ServerConfig serverConfig;

@Inject
public PlannerFactory(
final SchemaPlus rootSchema,
final QuerySegmentWalker walker,
final DruidOperatorTable operatorTable,
final PlannerConfig plannerConfig
final PlannerConfig plannerConfig,
final ServerConfig serverConfig
)
{
this.rootSchema = rootSchema;
this.walker = walker;
this.operatorTable = operatorTable;
this.plannerConfig = plannerConfig;
this.serverConfig = serverConfig;
}

public DruidPlanner createPlanner(final Map<String, Object> queryContext)
{
final PlannerContext plannerContext = PlannerContext.create(plannerConfig, queryContext);
final QueryMaker queryMaker = new QueryMaker(walker, plannerContext);
final QueryMaker queryMaker = new QueryMaker(walker, plannerContext, serverConfig);
final FrameworkConfig frameworkConfig = Frameworks
.newConfigBuilder()
.parserConfig(
Expand Down
71 changes: 55 additions & 16 deletions sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,9 @@
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.primitives.Doubles;
import com.google.common.primitives.Ints;
import io.druid.client.DirectDruidClient;
import io.druid.common.guava.GuavaUtils;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Sequence;
Expand All @@ -47,6 +47,7 @@
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultValue;
import io.druid.segment.column.Column;
import io.druid.server.initialization.ServerConfig;
import io.druid.sql.calcite.planner.Calcites;
import io.druid.sql.calcite.planner.PlannerContext;
import io.druid.sql.calcite.table.RowSignature;
Expand All @@ -69,14 +70,17 @@ public class QueryMaker
{
private final QuerySegmentWalker walker;
private final PlannerContext plannerContext;
private final ServerConfig serverConfig;

public QueryMaker(
final QuerySegmentWalker walker,
final PlannerContext plannerContext
final PlannerContext plannerContext,
final ServerConfig serverConfig
)
{
this.walker = walker;
this.plannerContext = plannerContext;
this.serverConfig = serverConfig;
}

public PlannerContext getPlannerContext()
Expand Down Expand Up @@ -179,12 +183,15 @@ public boolean hasNext()
@Override
public Sequence<Object[]> next()
{
final SelectQuery queryWithPagination = baseQuery.withPagingSpec(
new PagingSpec(
pagingIdentifiers.get(),
plannerContext.getPlannerConfig().getSelectThreshold(),
true
)
final SelectQuery queryWithPagination = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
baseQuery.withPagingSpec(
new PagingSpec(
pagingIdentifiers.get(),
plannerContext.getPlannerConfig().getSelectThreshold(),
true
)
),
serverConfig
);

Hook.QUERY_PLAN.run(queryWithPagination);
Expand All @@ -194,7 +201,13 @@ public Sequence<Object[]> next()

return Sequences.concat(
Sequences.map(
QueryPlus.wrap(queryWithPagination).run(walker, Maps.<String, Object>newHashMap()),
QueryPlus.wrap(queryWithPagination)
.run(walker,
DirectDruidClient.makeResponseContextForQuery(
queryWithPagination,
plannerContext.getQueryStartTimeMillis()
)
),
new Function<Result<SelectResultValue>, Sequence<Object[]>>()
{
@Override
Expand Down Expand Up @@ -255,17 +268,26 @@ public void remove()

private Sequence<Object[]> executeTimeseries(
final DruidQueryBuilder queryBuilder,
final TimeseriesQuery query
final TimeseriesQuery baseQuery
)
{
final TimeseriesQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
baseQuery,
serverConfig
);

final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();
final List<DimensionSpec> dimensions = queryBuilder.getGrouping().getDimensions();
final String timeOutputName = dimensions.isEmpty() ? null : Iterables.getOnlyElement(dimensions).getOutputName();

Hook.QUERY_PLAN.run(query);

return Sequences.map(
QueryPlus.wrap(query).run(walker, Maps.<String, Object>newHashMap()),
QueryPlus.wrap(query)
.run(
walker,
DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis())
),
new Function<Result<TimeseriesResultValue>, Object[]>()
{
@Override
Expand All @@ -291,16 +313,25 @@ public Object[] apply(final Result<TimeseriesResultValue> result)

private Sequence<Object[]> executeTopN(
final DruidQueryBuilder queryBuilder,
final TopNQuery query
final TopNQuery baseQuery
)
{
final TopNQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
baseQuery,
serverConfig
);

final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();

Hook.QUERY_PLAN.run(query);

return Sequences.concat(
Sequences.map(
QueryPlus.wrap(query).run(walker, Maps.<String, Object>newHashMap()),
QueryPlus.wrap(query)
.run(
walker,
DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis())
),
new Function<Result<TopNResultValue>, Sequence<Object[]>>()
{
@Override
Expand Down Expand Up @@ -328,15 +359,23 @@ public Sequence<Object[]> apply(final Result<TopNResultValue> result)

private Sequence<Object[]> executeGroupBy(
final DruidQueryBuilder queryBuilder,
final GroupByQuery query
final GroupByQuery baseQuery
)
{
final GroupByQuery query = DirectDruidClient.withDefaultTimeoutAndMaxScatterGatherBytes(
baseQuery,
serverConfig
);

final List<RelDataTypeField> fieldList = queryBuilder.getRowType().getFieldList();

Hook.QUERY_PLAN.run(query);

return Sequences.map(
QueryPlus.wrap(query).run(walker, Maps.<String, Object>newHashMap()),
QueryPlus.wrap(query)
.run(
walker,
DirectDruidClient.makeResponseContextForQuery(query, plannerContext.getQueryStartTimeMillis())
),
new Function<io.druid.data.input.Row, Object[]>()
{
@Override
Expand Down
Loading