Make subquery IDs more comprehensive (#11809)

There are 3 types of query IDs - id, subQueryId, sqlQueryId. Currently, whenever a query generates subqueries, the subquery's subQueryId is populated randomly. Also, subquery's Id is not set to the parent query Id. Therefore there is no way of linking the subqueries to the parent query, and one loses the ability to look at end to end view of the query.

This PR aims to implement following couple of things:

Populate the subqueries with it's parent's id (and sqlQueryId if present)
Populate the subqueryId such that it forms a hierarchical relationship amongs themselves. For example, if there is a query which launches a subquery, which in turn launches a couple of subqueries, then the ids and subQueryIds should have following structure.
This commit is contained in:
Laksh Singla 2021-11-11 16:31:56 +05:30 committed by GitHub
parent f9941c12c3
commit 57ed5127a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 200 additions and 59 deletions

View File

@ -138,6 +138,7 @@ public interface Query<T>
*/
Query<T> withSubQueryId(String subQueryId);
@SuppressWarnings("unused")
default Query<T> withDefaultSubQueryId()
{
return withSubQueryId(UUID.randomUUID().toString());

View File

@ -22,13 +22,18 @@ package org.apache.druid.query;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.guava.MergeSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.planning.DataSourceAnalysis;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class UnionQueryRunner<T> implements QueryRunner<T>
{
private final QueryRunner<T> baseRunner;
@ -72,19 +77,26 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
query.getResultOrdering(),
Sequences.simple(
Lists.transform(
unionDataSource.getDataSources(),
(Function<DataSource, Sequence<T>>) singleSource ->
IntStream.range(0, unionDataSource.getDataSources().size())
.mapToObj(i -> new Pair<>(unionDataSource.getDataSources().get(i), i + 1))
.collect(Collectors.toList()),
(Function<Pair<TableDataSource, Integer>, Sequence<T>>) singleSourceWithIndex ->
baseRunner.run(
queryPlus.withQuery(
Queries.withBaseDataSource(query, singleSource)
Queries.withBaseDataSource(query, singleSourceWithIndex.lhs)
// assign the subqueryId. this will be used to validate that every query servers
// have responded per subquery in RetryQueryRunner
.withDefaultSubQueryId()
.withSubQueryId(generateSubqueryId(
query.getSubQueryId(),
singleSourceWithIndex.lhs.getName(),
singleSourceWithIndex.rhs
))
),
responseContext
)
)
)
);
}
} else {
@ -93,4 +105,21 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
}
}
/**
* Appends and returns the name and the position of the individual datasource in the union with the parent query id
* if preseent
*
* @param parentSubqueryId The subquery Id of the parent query which is generating this subquery
* @param dataSourceName Name of the datasource for which the UnionRunner is running
* @param dataSourceIndex Position of the datasource for which the UnionRunner is running
* @return Subquery Id which needs to be populated
*/
private String generateSubqueryId(String parentSubqueryId, String dataSourceName, int dataSourceIndex)
{
String dataSourceNameIndex = dataSourceName + "." + dataSourceIndex;
if (StringUtils.isEmpty(parentSubqueryId)) {
return dataSourceNameIndex;
}
return parentSubqueryId + "." + dataSourceNameIndex;
}
}

View File

@ -23,11 +23,13 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.client.CachingClusteredClient;
import org.apache.druid.client.DirectDruidClient;
import org.apache.druid.client.cache.Cache;
import org.apache.druid.client.cache.CacheConfig;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
@ -58,9 +60,13 @@ import org.apache.druid.server.initialization.ServerConfig;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Stack;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
@ -147,9 +153,18 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
// transform TableDataSource to GlobalTableDataSource when eligible
// before further transformation to potentially inline
final DataSource freeTradeDataSource = globalizeIfPossible(query.getDataSource());
// Populate the subquery ids of the subquery id present in the main query
Query<T> newQuery = query.withDataSource(generateSubqueryIds(
query.getDataSource(),
query.getId(),
query.getSqlQueryId()
));
final DataSource freeTradeDataSource = globalizeIfPossible(newQuery.getDataSource());
// do an inlining dry run to see if any inlining is necessary, without actually running the queries.
final int maxSubqueryRows = QueryContexts.getMaxSubqueryRows(query, serverConfig.getMaxSubqueryRows());
final DataSource inlineDryRun = inlineIfNecessary(
freeTradeDataSource,
toolChest,
@ -165,7 +180,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
}
// Now that we know the structure is workable, actually do the inlining (if necessary).
final Query<T> newQuery = query.withDataSource(
newQuery = newQuery.withDataSource(
inlineIfNecessary(
freeTradeDataSource,
toolChest,
@ -287,6 +302,9 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
* {@link QueryToolChest#canPerformSubquery}. If the toolchest can handle it, then it will
* not be inlined. See {@link org.apache.druid.query.groupby.GroupByQueryQueryToolChest}
* for an example of a toolchest that can handle subqueries.
* @param subqueryRowLimitAccumulator an accumulator for tracking the number of accumulated rows in all subqueries
* for a particular master query
* @param maxSubqueryRows Max rows that all the subqueries generated by a master query can have, combined
* @param dryRun if true, does not actually execute any subqueries, but will inline empty result sets.
*/
@SuppressWarnings({"rawtypes", "unchecked"}) // Subquery, toolchest, runner handling all use raw types
@ -332,24 +350,23 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
}
} else if (canRunQueryUsingLocalWalker(subQuery) || canRunQueryUsingClusterWalker(subQuery)) {
// Subquery needs to be inlined. Assign it a subquery id and run it.
final Query subQueryWithId = subQuery.withDefaultSubQueryId();
final Sequence<?> queryResults;
if (dryRun) {
queryResults = Sequences.empty();
} else {
final QueryRunner subqueryRunner = subQueryWithId.getRunner(this);
final QueryRunner subqueryRunner = subQuery.getRunner(this);
queryResults = subqueryRunner.run(
QueryPlus.wrap(subQueryWithId),
QueryPlus.wrap(subQuery),
DirectDruidClient.makeResponseContextForQuery()
);
}
return toInlineDataSource(
subQueryWithId,
subQuery,
queryResults,
warehouse.getToolChest(subQueryWithId),
warehouse.getToolChest(subQuery),
subqueryRowLimitAccumulator,
maxSubqueryRows
);
@ -431,6 +448,100 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
);
}
/**
* This method returns the datasource by populating all the {@link QueryDataSource} with correct nesting level and
* sibling order of all the subqueries that are present.
* It also plumbs parent query's id and sql id in case the subqueries don't have it set by default
*
* @param rootDataSource Datasource whose subqueries need to be populated
* @param parentQueryId Parent Query's ID, can be null if do not need to update this in the subqueries
* @param parentSqlQueryId Parent Query's SQL Query ID, can be null if do not need to update this in the subqueries
* @return DataSource populated with the subqueries
*/
private DataSource generateSubqueryIds(
DataSource rootDataSource,
@Nullable final String parentQueryId,
@Nullable final String parentSqlQueryId
)
{
Queue<DataSource> queue = new ArrayDeque<>();
queue.add(rootDataSource);
// Performs BFS on the datasource tree to find the nesting level, and the sibling order of the query datasource
Map<QueryDataSource, Pair<Integer, Integer>> queryDataSourceToSubqueryIds = new HashMap<>();
int level = 1;
while (!queue.isEmpty()) {
int size = queue.size();
int siblingOrder = 1;
for (int i = 0; i < size; ++i) {
DataSource currentDataSource = queue.poll();
if (currentDataSource == null) { // Shouldn't be encountered
continue;
}
if (currentDataSource instanceof QueryDataSource) {
queryDataSourceToSubqueryIds.put((QueryDataSource) currentDataSource, new Pair<>(level, siblingOrder));
++siblingOrder;
}
queue.addAll(currentDataSource.getChildren());
}
++level;
}
/*
Returns the datasource by populating all the subqueries with the id generated in the map above.
Implemented in a separate function since the methods on datasource and queries return a new datasource/query
*/
return insertSubqueryIds(rootDataSource, queryDataSourceToSubqueryIds, parentQueryId, parentSqlQueryId);
}
/**
* To be used in conjunction with {@code generateSubqueryIds()} method. This does the actual task of populating the
* query's id, subQueryId and sqlQueryId
*
* @param currentDataSource The datasource to be populated with the subqueries
* @param queryDataSourceToSubqueryIds Map of the datasources to their level and sibling order
* @param parentQueryId Parent query's id
* @param parentSqlQueryId Parent query's sqlQueryId
* @return Populates the subqueries from the map
*/
private DataSource insertSubqueryIds(
DataSource currentDataSource,
Map<QueryDataSource, Pair<Integer, Integer>> queryDataSourceToSubqueryIds,
@Nullable final String parentQueryId,
@Nullable final String parentSqlQueryId
)
{
if (currentDataSource instanceof QueryDataSource
&& queryDataSourceToSubqueryIds.containsKey((QueryDataSource) currentDataSource)) {
QueryDataSource queryDataSource = (QueryDataSource) currentDataSource;
Pair<Integer, Integer> nestingInfo = queryDataSourceToSubqueryIds.get(queryDataSource);
String subQueryId = nestingInfo.lhs + "." + nestingInfo.rhs;
Query<?> query = queryDataSource.getQuery();
if (StringUtils.isEmpty(query.getSubQueryId())) {
query = query.withSubQueryId(subQueryId);
}
if (StringUtils.isEmpty(query.getId()) && StringUtils.isNotEmpty(parentQueryId)) {
query = query.withId(parentQueryId);
}
if (StringUtils.isEmpty(query.getSqlQueryId()) && StringUtils.isNotEmpty(parentSqlQueryId)) {
query = query.withSqlQueryId(parentSqlQueryId);
}
currentDataSource = new QueryDataSource(query);
}
return currentDataSource.withChildren(currentDataSource.getChildren()
.stream()
.map(childDataSource -> insertSubqueryIds(
childDataSource,
queryDataSourceToSubqueryIds,
parentQueryId,
parentSqlQueryId
))
.collect(Collectors.toList()));
}
/**
* Convert the results of a particular query into a materialized (List-based) InlineDataSource.
*
@ -441,7 +552,6 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
* particular master query
* @param limit user-configured limit. If negative, will be treated as {@link Integer#MAX_VALUE}.
* If zero, this method will throw an error immediately.
*
* @throws ResourceLimitExceededException if the limit is exceeded
*/
private static <T, QueryType extends Query<T>> InlineDataSource toInlineDataSource(

View File

@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.BaseQuery;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Druids;
import org.apache.druid.query.GlobalTableDataSource;
@ -102,7 +101,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.UUID;
/**
* Tests ClientQuerySegmentWalker.
@ -203,6 +201,8 @@ public class ClientQuerySegmentWalkerTest
.build()
);
private static final String DUMMY_QUERY_ID = "dummyQueryId";
@Rule
public ExpectedException expectedException = ExpectedException.none();
@ -249,7 +249,7 @@ public class ClientQuerySegmentWalkerTest
.aggregators(new LongSumAggregatorFactory("sum", "n"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
@ -274,7 +274,7 @@ public class ClientQuerySegmentWalkerTest
.aggregators(new LongSumAggregatorFactory("sum", "n"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// expect global/joinable datasource to be automatically translated into a GlobalTableDataSource
final TimeseriesQuery expectedClusterQuery =
@ -285,7 +285,7 @@ public class ClientQuerySegmentWalkerTest
.aggregators(new LongSumAggregatorFactory("sum", "n"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
testQuery(
query,
@ -309,7 +309,7 @@ public class ClientQuerySegmentWalkerTest
.intervals(Collections.singletonList(INTERVAL))
.aggregators(new LongSumAggregatorFactory("sum", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
@ -341,12 +341,12 @@ public class ClientQuerySegmentWalkerTest
.intervals(Intervals.ONLY_ETERNITY)
.aggregators(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
ImmutableList.of(
ExpectedQuery.cluster(subquery),
ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")),
ExpectedQuery.local(
query.withDataSource(
InlineDataSource.fromIterable(
@ -386,12 +386,12 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Intervals.ONLY_ETERNITY)
.setAggregatorSpecs(new CountAggregatorFactory("cnt"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
testQuery(
query,
// GroupBy handles its own subqueries; only the inner one will go to the cluster.
ImmutableList.of(ExpectedQuery.cluster(subquery)),
// GroupBy handles its own subqueries; only the inner one will go to the cluster. Also, it gets a subquery id
ImmutableList.of(ExpectedQuery.cluster(subquery.withSubQueryId("1.1"))),
ImmutableList.of(new Object[]{3L})
);
@ -419,13 +419,13 @@ public class ClientQuerySegmentWalkerTest
.setDimensions(DefaultDimensionSpec.of("s"))
.setAggregatorSpecs(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
ImmutableList.of(
ExpectedQuery.cluster(query.withDataSource(new TableDataSource(FOO))),
ExpectedQuery.cluster(query.withDataSource(new TableDataSource(BAR)))
ExpectedQuery.cluster(query.withDataSource(new TableDataSource(FOO)).withSubQueryId("foo.1")),
ExpectedQuery.cluster(query.withDataSource(new TableDataSource(BAR)).withSubQueryId("bar.2"))
),
ImmutableList.of(
new Object[]{"a", 2L},
@ -458,7 +458,7 @@ public class ClientQuerySegmentWalkerTest
.setDimensions(DefaultDimensionSpec.of("s"))
.setAggregatorSpecs(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
@ -508,12 +508,12 @@ public class ClientQuerySegmentWalkerTest
.setDimensions(DefaultDimensionSpec.of("s"), DefaultDimensionSpec.of("j.s"))
.setAggregatorSpecs(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
ImmutableList.of(
ExpectedQuery.cluster(subquery),
ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("2.1")),
ExpectedQuery.cluster(
query.withDataSource(
query.getDataSource().withChildren(
@ -576,20 +576,22 @@ public class ClientQuerySegmentWalkerTest
.setDimensions(DefaultDimensionSpec.of("s"), DefaultDimensionSpec.of("j.s"))
.setAggregatorSpecs(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
ImmutableList.of(
ExpectedQuery.cluster(
subquery.withDataSource(
subquery.getDataSource().getChildren().get(0)
)
subquery
.withDataSource(subquery.getDataSource().getChildren().get(0))
.withId(DUMMY_QUERY_ID)
.withSubQueryId("2.1.foo.1")
),
ExpectedQuery.cluster(
subquery.withDataSource(
subquery.getDataSource().getChildren().get(1)
)
subquery
.withDataSource(subquery.getDataSource().getChildren().get(1))
.withId(DUMMY_QUERY_ID)
.withSubQueryId("2.1.bar.2")
),
ExpectedQuery.cluster(
query.withDataSource(
@ -602,7 +604,7 @@ public class ClientQuerySegmentWalkerTest
)
)
)
)
).withSubQueryId("foo.1")
),
ExpectedQuery.cluster(
query.withDataSource(
@ -615,7 +617,7 @@ public class ClientQuerySegmentWalkerTest
)
)
)
)
).withSubQueryId("bar.2")
)
),
ImmutableList.of(new Object[]{"y", "y", 1L})
@ -650,13 +652,13 @@ public class ClientQuerySegmentWalkerTest
.setDimensions(DefaultDimensionSpec.of("s"))
.setAggregatorSpecs(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
// GroupBy handles its own subqueries; only the inner one will go to the cluster.
ImmutableList.of(
ExpectedQuery.cluster(subquery),
ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")),
ExpectedQuery.local(
query.withDataSource(
InlineDataSource.fromIterable(
@ -706,13 +708,13 @@ public class ClientQuerySegmentWalkerTest
.threshold(100)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
testQuery(
query,
// GroupBy handles its own subqueries; only the inner one will go to the cluster.
ImmutableList.of(
ExpectedQuery.cluster(subquery),
ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")),
ExpectedQuery.local(
query.withDataSource(
InlineDataSource.fromIterable(
@ -761,7 +763,7 @@ public class ClientQuerySegmentWalkerTest
.setDimensions(DefaultDimensionSpec.of("s"), DefaultDimensionSpec.of("j.s"))
.setAggregatorSpecs(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
expectedException.expect(IllegalStateException.class);
expectedException.expectMessage("Cannot handle subquery structure for dataSource");
@ -789,7 +791,7 @@ public class ClientQuerySegmentWalkerTest
.intervals(Intervals.ONLY_ETERNITY)
.aggregators(new CountAggregatorFactory("cnt"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
expectedException.expect(ResourceLimitExceededException.class);
expectedException.expectMessage("Subquery generated results beyond maximum[2]");
@ -807,7 +809,7 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("ad"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace
@ -831,7 +833,7 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("ad"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky
@ -865,7 +867,7 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("al"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace
expectedException.expect(RuntimeException.class);
@ -888,7 +890,7 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("al"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky
@ -922,7 +924,7 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("as"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace
@ -946,7 +948,7 @@ public class ClientQuerySegmentWalkerTest
.setInterval(Collections.singletonList(INTERVAL))
.setDimensions(DefaultDimensionSpec.of("as"))
.build()
.withId("queryId");
.withId(DUMMY_QUERY_ID);
// 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky
@ -983,7 +985,7 @@ public class ClientQuerySegmentWalkerTest
.threshold(1000)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace
@ -1009,7 +1011,7 @@ public class ClientQuerySegmentWalkerTest
.threshold(1000)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky
@ -1045,7 +1047,7 @@ public class ClientQuerySegmentWalkerTest
.threshold(1000)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace
expectedException.expect(RuntimeException.class);
@ -1070,7 +1072,7 @@ public class ClientQuerySegmentWalkerTest
.threshold(1000)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky
@ -1106,7 +1108,7 @@ public class ClientQuerySegmentWalkerTest
.threshold(1000)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace
@ -1132,7 +1134,7 @@ public class ClientQuerySegmentWalkerTest
.threshold(1000)
.aggregators(new LongSumAggregatorFactory("sum_n", "n"))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky
@ -1167,7 +1169,7 @@ public class ClientQuerySegmentWalkerTest
.aggregators(new LongSumAggregatorFactory("sum", "al"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// sum doesn't know what to do with an array, so gets 0
testQuery(
@ -1193,7 +1195,7 @@ public class ClientQuerySegmentWalkerTest
.aggregators(new LongSumAggregatorFactory("sum", "al"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId(UUID.randomUUID().toString());
.withId(DUMMY_QUERY_ID);
// sum doesn't know what to do with an array also if type is null, so gets 0
testQuery(
@ -1363,7 +1365,6 @@ public class ClientQuerySegmentWalkerTest
// Need to blast various parameters that will vary and aren't important to test for.
this.query = query.withOverriddenContext(
ImmutableMap.<String, Object>builder()
.put(BaseQuery.SUB_QUERY_ID, "dummy")
.put(DirectDruidClient.QUERY_FAIL_TIME, 0L)
.put(QueryContexts.DEFAULT_TIMEOUT_KEY, 0L)
.put(QueryContexts.FINALIZE_KEY, true)