Remove QueryRunner.run(Query, responseContext) and related legacy methods (#4482)

* Remove QueryRunner.run(Query, responseContext) and related legacy methods

* Remove local var
This commit is contained in:
Roman Leventov 2017-08-11 03:12:38 +03:00 committed by Jihoon Son
parent c821bc9a5a
commit bf28d0775b
66 changed files with 735 additions and 794 deletions

View File

@ -38,6 +38,7 @@ import io.druid.js.JavaScriptConfig;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -251,7 +252,7 @@ public class FilteredAggregatorBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}

View File

@ -49,6 +49,7 @@ import io.druid.offheap.OffheapBufferGenerator;
import io.druid.query.DruidProcessingConfig;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -472,7 +473,7 @@ public class GroupByTypeInterfaceBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}

View File

@ -39,6 +39,7 @@ import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -327,7 +328,7 @@ public class TopNTypeInterfaceBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}

View File

@ -50,6 +50,7 @@ import io.druid.offheap.OffheapBufferGenerator;
import io.druid.query.DruidProcessingConfig;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -572,7 +573,7 @@ public class GroupByBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@ -625,7 +626,7 @@ public class GroupByBenchmark
(QueryToolChest) toolChest
);
Sequence<Row> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
List<Row> results = Sequences.toList(queryResult, Lists.<Row>newArrayList());
for (Row result : results) {
@ -649,7 +650,7 @@ public class GroupByBenchmark
final GroupByQuery spillingQuery = query.withOverriddenContext(
ImmutableMap.<String, Object>of("bufferGrouperMaxSize", 4000)
);
Sequence<Row> queryResult = theRunner.run(spillingQuery, Maps.<String, Object>newHashMap());
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), Maps.newHashMap());
List<Row> results = Sequences.toList(queryResult, Lists.<Row>newArrayList());
for (Row result : results) {
@ -676,7 +677,7 @@ public class GroupByBenchmark
(QueryToolChest) toolChest
);
Sequence<Row> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
List<Row> results = Sequences.toList(queryResult, Lists.<Row>newArrayList());
for (Row result : results) {

View File

@ -42,6 +42,7 @@ import io.druid.query.Druids;
import io.druid.query.Druids.SearchQueryBuilder;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -400,7 +401,7 @@ public class SearchBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@ -465,7 +466,10 @@ public class SearchBenchmark
)
);
Sequence<Result<SearchResultValue>> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<Result<SearchResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(query),
Maps.<String, Object>newHashMap()
);
List<Result<SearchResultValue>> results = Sequences.toList(
queryResult,
Lists.<Result<SearchResultValue>>newArrayList()

View File

@ -40,6 +40,7 @@ import io.druid.java.util.common.logger.Logger;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -265,7 +266,7 @@ public class SelectBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@ -374,7 +375,7 @@ public class SelectBenchmark
boolean done = false;
while (!done) {
Sequence<Result<SelectResultValue>> queryResult = theRunner.run(queryCopy, Maps.<String, Object>newHashMap());
Sequence<Result<SelectResultValue>> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), Maps.newHashMap());
List<Result<SelectResultValue>> results = Sequences.toList(queryResult, Lists.<Result<SelectResultValue>>newArrayList());
SelectResultValue result = results.get(0).getValue();

View File

@ -31,6 +31,7 @@ import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunnerFactoryConglomerate;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
@ -165,7 +166,7 @@ public class SqlBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryNative(Blackhole blackhole) throws Exception
{
final Sequence<Row> resultSequence = groupByQuery.run(walker, Maps.<String, Object>newHashMap());
final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, Maps.newHashMap());
final ArrayList<Row> resultList = Sequences.toList(resultSequence, Lists.<Row>newArrayList());
for (Row row : resultList) {

View File

@ -37,6 +37,7 @@ import io.druid.java.util.common.logger.Logger;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -324,7 +325,7 @@ public class TimeseriesBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@ -406,7 +407,10 @@ public class TimeseriesBenchmark
)
);
Sequence<Result<TimeseriesResultValue>> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<Result<TimeseriesResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(query),
Maps.<String, Object>newHashMap()
);
List<Result<TimeseriesResultValue>> results = Sequences.toList(queryResult, Lists.<Result<TimeseriesResultValue>>newArrayList());
for (Result<TimeseriesResultValue> result : results) {

View File

@ -38,6 +38,7 @@ import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -305,7 +306,7 @@ public class TopNBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@ -367,7 +368,10 @@ public class TopNBenchmark
)
);
Sequence<Result<TopNResultValue>> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<Result<TopNResultValue>> queryResult = theRunner.run(
QueryPlus.wrap(query),
Maps.<String, Object>newHashMap()
);
List<Result<TopNResultValue>> results = Sequences.toList(queryResult, Lists.<Result<TopNResultValue>>newArrayList());
for (Result<TopNResultValue> result : results) {

View File

@ -195,7 +195,7 @@ public class MultiSegmentScanQueryTest
factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of(
factory.createRunner(segment0),
factory.createRunner(segment1)
)).run(query, new HashMap<String, Object>()),
)).run(QueryPlus.wrap(query), new HashMap<String, Object>()),
Lists.<ScanResultValue>newArrayList()
);
int totalCount = 0;
@ -232,7 +232,7 @@ public class MultiSegmentScanQueryTest
);
ScanQuery query = newBuilder().build();
List<ScanResultValue> results = Sequences.toList(
runner.run(query, new HashMap<String, Object>()),
runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()),
Lists.<ScanResultValue>newArrayList()
);
int totalCount = 0;

View File

@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.DefaultGenericQueryMetricsFactory;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.TableDataSource;
@ -159,7 +160,7 @@ public class ScanQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<ScanResultValue>newArrayList()
);
@ -202,7 +203,7 @@ public class ScanQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<ScanResultValue>newArrayList()
);
@ -225,7 +226,7 @@ public class ScanQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<ScanResultValue>newArrayList()
);
@ -263,7 +264,7 @@ public class ScanQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<ScanResultValue>newArrayList()
);
@ -304,7 +305,7 @@ public class ScanQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<ScanResultValue>newArrayList()
);
@ -366,13 +367,13 @@ public class ScanQueryRunnerTest
.build();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<ScanResultValue>newArrayList()
);
Iterable<ScanResultValue> resultsOptimize = Sequences.toList(
toolChest
.postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)))
.run(query, Maps.<String, Object>newHashMap()), Lists.<ScanResultValue>newArrayList()
.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap()), Lists.<ScanResultValue>newArrayList()
);
final List<List<Map<String, Object>>> events = toEvents(
@ -422,7 +423,7 @@ public class ScanQueryRunnerTest
.build();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<ScanResultValue>newArrayList()
);
@ -440,7 +441,7 @@ public class ScanQueryRunnerTest
.build();
Iterable<ScanResultValue> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<ScanResultValue>newArrayList()
);

View File

@ -32,6 +32,7 @@ import io.druid.data.input.impl.TimestampSpec;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -185,7 +186,7 @@ public class MapVirtualColumnTest
private void checkSelectQuery(SelectQuery searchQuery, List<Map> expected) throws Exception
{
List<Result<SelectResultValue>> results = Sequences.toList(
runner.run(searchQuery, ImmutableMap.of()),
runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
Assert.assertEquals(1, results.size());

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.collections.StupidPool;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -247,6 +248,6 @@ public class ApproximateHistogramTopNQueryTest
);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
}

View File

@ -87,6 +87,7 @@ import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
import io.druid.query.Druids;
import io.druid.query.IntervalChunkingQueryRunnerDecorator;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
@ -1728,7 +1729,7 @@ public class KafkaIndexTaskTest
.build();
ArrayList<Result<TimeseriesResultValue>> results = Sequences.toList(
task.getQueryRunner(query).run(query, ImmutableMap.<String, Object>of()),
task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.<String, Object>of()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);

View File

@ -20,9 +20,9 @@
package io.druid.query.aggregation.variance;
import com.google.common.collect.Lists;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
@ -107,7 +107,7 @@ public class VarianceTimeseriesQueryTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, new HashMap<String, Object>()),
runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);

View File

@ -22,8 +22,8 @@ package io.druid.query.aggregation.variance;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.java.util.common.guava.Sequence;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -142,7 +142,10 @@ public class VarianceTopNQueryTest
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
);
final QueryRunner<Result<TopNResultValue>> mergeRunner = chest.mergeResults(runner);
final Sequence<Result<TopNResultValue>> retval = mergeRunner.run(query, ImmutableMap.<String, Object>of());
final Sequence<Result<TopNResultValue>> retval = mergeRunner.run(
QueryPlus.wrap(query),
ImmutableMap.<String, Object>of()
);
TestHelper.assertExpectedResults(expectedResults, retval);
return retval;
}

View File

@ -77,6 +77,7 @@ import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
import io.druid.query.Druids;
import io.druid.query.IntervalChunkingQueryRunnerDecorator;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerFactoryConglomerate;
@ -1067,7 +1068,7 @@ public class RealtimeIndexTaskTest
.build();
ArrayList<Result<TimeseriesResultValue>> results = Sequences.toList(
task.getQueryRunner(query).run(query, ImmutableMap.<String, Object>of()),
task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.<String, Object>of()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric(metric);

View File

@ -24,7 +24,6 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import io.druid.java.util.common.guava.Sequence;
import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.Duration;
import org.joda.time.Interval;
@ -87,15 +86,9 @@ public abstract class BaseQuery<T extends Comparable<T>> implements Query<T>
}
@Override
public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context)
public QueryRunner<T> getRunner(QuerySegmentWalker walker, Map<String, Object> context)
{
return run(querySegmentSpec.lookup(this, walker), context);
}
@Override
public Sequence<T> run(QueryRunner<T> runner, Map<String, Object> context)
{
return runner.run(this, context);
return querySegmentSpec.lookup(this, walker);
}
@Override

View File

@ -22,7 +22,6 @@ package io.druid.query;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.collect.Ordering;
import io.druid.java.util.common.guava.Sequence;
import io.druid.query.datasourcemetadata.DataSourceMetadataQuery;
import io.druid.query.filter.DimFilter;
import io.druid.query.groupby.GroupByQuery;
@ -70,20 +69,7 @@ public interface Query<T>
String getType();
/**
* @deprecated use {@link QueryPlus#run(QuerySegmentWalker, Map)} instead. This method is going to be removed in Druid
* 0.11. In the future, a method like getRunner(QuerySegmentWalker, Map) could be added instead of this method, so
* that {@link QueryPlus#run(QuerySegmentWalker, Map)} could be implemented as {@code
* this.query.getRunner(walker, context).run(this, context))}.
*/
@Deprecated
Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context);
/**
* @deprecated use {@link QueryRunner#run(QueryPlus, Map)} instead. This method is going to be removed in Druid 0.11.
*/
@Deprecated
Sequence<T> run(QueryRunner<T> runner, Map<String, Object> context);
QueryRunner<T> getRunner(QuerySegmentWalker walker, Map<String, Object> context);
List<Interval> getIntervals();

View File

@ -124,11 +124,6 @@ public final class QueryPlus<T>
public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context)
{
if (query instanceof BaseQuery) {
return ((BaseQuery) query).getQuerySegmentSpec().lookup(query, walker).run(this, context);
} else {
// fallback
return query.run(walker, context);
}
return query.getRunner(walker, context).run(this, context);
}
}

View File

@ -23,27 +23,10 @@ import io.druid.java.util.common.guava.Sequence;
import java.util.Map;
/**
* This interface has two similar run() methods. {@link #run(Query, Map)} is legacy and {@link #run(QueryPlus, Map)}
* is the new one. Their default implementations delegate to each other. Every implementation of QueryRunner should
* override only one of those methods. New implementations should override the new method: {@link #run(QueryPlus, Map)}.
*/
public interface QueryRunner<T>
{
/**
* @deprecated use and override {@link #run(QueryPlus, Map)} instead. This method is going to be removed in Druid 0.11
*/
@Deprecated
default Sequence<T> run(Query<T> query, Map<String, Object> responseContext)
{
return run(QueryPlus.wrap(query), responseContext);
}
/**
* Runs the given query and returns results in a time-ordered sequence.
*/
default Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext)
{
return run(queryPlus.getQuery(), responseContext);
}
Sequence<T> run(QueryPlus<T> queryPlus, Map<String, Object> responseContext);
}

View File

@ -80,7 +80,7 @@ public class AsyncQueryRunnerTest
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
Sequence lazy = asyncRunner.run(query, Collections.EMPTY_MAP);
Sequence lazy = asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
latch.countDown();
Assert.assertEquals(Lists.newArrayList(1), Sequences.toList(lazy, Lists.newArrayList()));
}
@ -110,7 +110,7 @@ public class AsyncQueryRunnerTest
);
Sequence lazy = asyncRunner.run(
query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1)),
QueryPlus.wrap(query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 1))),
Collections.EMPTY_MAP
);
@ -127,14 +127,7 @@ public class AsyncQueryRunnerTest
@Test
public void testQueryRegistration()
{
QueryRunner baseRunner = new QueryRunner()
{
@Override
public Sequence run(QueryPlus queryPlus, Map responseContext)
{
return null;
}
};
QueryRunner baseRunner = (queryPlus, responseContext) -> null;
QueryWatcher mock = EasyMock.createMock(QueryWatcher.class);
mock.registerQuery(EasyMock.eq(query), EasyMock.anyObject(ListenableFuture.class));
@ -142,7 +135,7 @@ public class AsyncQueryRunnerTest
AsyncQueryRunner asyncRunner = new AsyncQueryRunner<>(baseRunner, executor, mock);
asyncRunner.run(query, Collections.EMPTY_MAP);
asyncRunner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
EasyMock.verify(mock);
}
}

View File

@ -26,8 +26,8 @@ import com.google.common.util.concurrent.ListenableFuture;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.lifecycle.Lifecycle;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.timeseries.TimeseriesQuery;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.easymock.IAnswer;
@ -119,14 +119,12 @@ public class ChainedExecutionQueryRunnerTest
)
);
Map<String, Object> context = ImmutableMap.<String, Object>of();
final Sequence seq = chainedRunner.run(
Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.intervals("2014/2015")
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
.build(),
context
);
.aggregators(Lists.newArrayList(new CountAggregatorFactory("count")))
.build();
final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context);
Future resultFuture = Executors.newFixedThreadPool(1).submit(
new Runnable()
@ -246,15 +244,13 @@ public class ChainedExecutionQueryRunnerTest
)
);
HashMap<String, Object> context = new HashMap<String, Object>();
final Sequence seq = chainedRunner.run(
Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.intervals("2014/2015")
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
.aggregators(Lists.newArrayList(new CountAggregatorFactory("count")))
.context(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 100, "queryId", "test"))
.build(),
context
);
.build();
final Sequence seq = chainedRunner.run(QueryPlus.wrap(query), context);
Future resultFuture = Executors.newFixedThreadPool(1).submit(
new Runnable()

View File

@ -86,7 +86,7 @@ public class IntervalChunkingQueryRunnerTest
EasyMock.replay(toolChest);
QueryRunner runner = decorator.decorate(baseRunner, toolChest);
runner.run(query, Collections.EMPTY_MAP);
runner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
EasyMock.verify(executors);
}
@ -103,7 +103,7 @@ public class IntervalChunkingQueryRunnerTest
EasyMock.replay(toolChest);
QueryRunner runner = decorator.decorate(baseRunner, toolChest);
runner.run(query, Collections.EMPTY_MAP);
runner.run(QueryPlus.wrap(query), Collections.EMPTY_MAP);
EasyMock.verify(executors);
}

View File

@ -1,64 +0,0 @@
/*
* 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.query;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
/**
* Tests that if a QueryRunner overrides a legacy {@link QueryRunner#run(Query, Map)} method, it still works. This
* test should be removed when {@link QueryRunner#run(Query, Map)} is removed.
*/
public class LegacyApiQueryRunnerTest
{
private static class LegacyApiQueryRunner<T> implements QueryRunner<T>
{
/**
* Overrides legacy API.
*/
@Override
public Sequence<T> run(Query<T> query, Map<String, Object> responseContext)
{
return Sequences.empty();
}
}
@Test
public void testQueryRunnerLegacyApi()
{
final Query query = new TestQuery(
new TableDataSource("test"),
new MultipleIntervalSegmentSpec(ImmutableList.of(new Interval("0/100"))),
false,
new HashMap()
);
Map<String, Object> context = new HashMap<>();
Assert.assertEquals(Sequences.empty(), new LegacyApiQueryRunner<>().run(QueryPlus.wrap(query), context));
}
}

View File

@ -310,7 +310,7 @@ public class MultiValuedDimensionTest
null
);
Map<String, Object> context = Maps.newHashMap();
Sequence<Result<TopNResultValue>> result = runner.run(query, context);
Sequence<Result<TopNResultValue>> result = runner.run(QueryPlus.wrap(query), context);
List<Result<TopNResultValue>> expectedResults = Arrays.asList(
new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),

View File

@ -129,7 +129,7 @@ public class RetryQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -189,7 +189,7 @@ public class RetryQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -248,7 +248,7 @@ public class RetryQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -292,7 +292,7 @@ public class RetryQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -396,7 +396,7 @@ public class RetryQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);

View File

@ -123,7 +123,7 @@ public class SchemaEvolutionTest
)
),
(QueryToolChest<T, Query<T>>) factory.getToolchest()
).run(query, Maps.<String, Object>newHashMap());
).run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(results, Lists.<T>newArrayList());
}

View File

@ -128,7 +128,10 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.<String, Object>of("metric", 5))
)
),
Sequences.toList(queryRunner.run(query, CONTEXT), Lists.<Result<TimeseriesResultValue>>newArrayList())
Sequences.toList(
queryRunner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
)
);
@ -178,7 +181,7 @@ public class TimewarpOperatorTest
)
),
Sequences.toList(
timeBoundaryRunner.run(timeBoundaryQuery, CONTEXT),
timeBoundaryRunner.run(QueryPlus.wrap(timeBoundaryQuery), CONTEXT),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
)
);
@ -233,7 +236,10 @@ public class TimewarpOperatorTest
new TimeseriesResultValue(ImmutableMap.<String, Object>of("metric", 3))
)
),
Sequences.toList(queryRunner.run(query, Maps.<String, Object>newHashMap()), Lists.<Result<TimeseriesResultValue>>newArrayList())
Sequences.toList(
queryRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
)
);
}
}

View File

@ -70,7 +70,7 @@ public class UnionQueryRunnerTest
.aggregators(QueryRunnerTestHelper.commonDoubleAggregators)
.build();
Map<String, Object> responseContext = Maps.newHashMap();
Sequence<?> result = runner.run(q, responseContext);
Sequence<?> result = runner.run(QueryPlus.wrap(q), responseContext);
List res = Sequences.toList(result, Lists.newArrayList());
Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), res);

View File

@ -551,7 +551,7 @@ public class AggregationTestHelper
toolChest
);
return baseRunner.run(query, Maps.newHashMap());
return baseRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
}
public QueryRunner<Row> makeStringSerdeQueryRunner(final ObjectMapper mapper, final QueryToolChest toolChest, final Query<Row> query, final QueryRunner<Row> baseRunner)

View File

@ -33,6 +33,7 @@ import io.druid.query.Druids;
import io.druid.query.GenericQueryMetricsFactory;
import io.druid.query.Query;
import io.druid.query.QueryContexts;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -137,7 +138,7 @@ public class DataSourceMetadataQueryTest
Map<String, Object> context = new MapMaker().makeMap();
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
Iterable<Result<DataSourceMetadataResultValue>> results = Sequences.toList(
runner.run(dataSourceMetadataQuery, context),
runner.run(QueryPlus.wrap(dataSourceMetadataQuery), context),
Lists.<Result<DataSourceMetadataResultValue>>newArrayList()
);
DataSourceMetadataResultValue val = results.iterator().next().getValue();

View File

@ -335,7 +335,7 @@ public class GroupByMultiSegmentTest
.setGranularity(Granularities.ALL)
.build();
Sequence<Row> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
List<Row> results = Sequences.toList(queryResult, Lists.<Row>newArrayList());
Row expectedRow = GroupByQueryRunnerTestHelper.createExpectedRow(

View File

@ -117,7 +117,7 @@ public class GroupByQueryRunnerFactoryTest
}
);
Sequence<Row> result = mergedRunner.run(query, Maps.newHashMap());
Sequence<Row> result = mergedRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
List<Row> expectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L),

View File

@ -30,7 +30,6 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.collections.BlockingPool;
import io.druid.collections.DefaultBlockingPool;
@ -54,14 +53,12 @@ import io.druid.query.ChainedExecutionQueryRunner;
import io.druid.query.DruidProcessingConfig;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryContexts;
import io.druid.query.QueryDataSource;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.QueryToolChest;
import io.druid.query.QueryWatcher;
import io.druid.query.ResourceLimitExceededException;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
@ -2523,7 +2520,7 @@ public class GroupByQueryRunnerTest
);
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
List<Row> allGranExpectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L),
@ -2537,7 +2534,11 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L)
);
TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged");
TestHelper.assertExpectedObjects(
allGranExpectedResults,
mergedRunner.run(QueryPlus.wrap(allGranQuery), context),
"merged"
);
}
@Test
@ -2582,7 +2583,9 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), StringUtils.format("limit: %d", limit)
Iterables.limit(expectedResults, limit),
mergeRunner.run(QueryPlus.wrap(fullQuery), context),
StringUtils.format("limit: %d", limit)
);
}
@ -2629,7 +2632,9 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), StringUtils.format("limit: %d", limit)
Iterables.limit(expectedResults, limit),
mergeRunner.run(QueryPlus.wrap(fullQuery), context),
StringUtils.format("limit: %d", limit)
);
}
@ -2684,7 +2689,9 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), StringUtils.format("limit: %d", limit)
Iterables.limit(expectedResults, limit),
mergeRunner.run(QueryPlus.wrap(fullQuery), context),
StringUtils.format("limit: %d", limit)
);
}
@ -2812,7 +2819,7 @@ public class GroupByQueryRunnerTest
);
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
}
@Test
@ -2849,10 +2856,12 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited"
Iterables.limit(expectedResults, 5),
mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context),
"limited"
);
// Now try it with an expression based aggregator.
@ -2877,9 +2886,14 @@ public class GroupByQueryRunnerTest
new Object[]{"2011-04-01", "mezzanine", 6L, 6635.47998046875D}
);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(builder.build(), context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited"
expectedResults,
mergeRunner.run(QueryPlus.wrap(builder.build()), context),
"no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5),
mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context),
"limited"
);
// Now try it with an expression virtual column.
@ -2894,9 +2908,15 @@ public class GroupByQueryRunnerTest
)
);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(builder.build(), context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited"
expectedResults,
mergeRunner.run(QueryPlus.wrap(builder.build()), context),
"no-limit"
);
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5),
mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context),
"limited"
);
}
@ -2934,9 +2954,11 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited"
Iterables.limit(expectedResults, 5),
mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context),
"limited"
);
}
@ -2975,9 +2997,11 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited"
Iterables.limit(expectedResults, 5),
mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context),
"limited"
);
}
@ -3015,9 +3039,11 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
TestHelper.assertExpectedObjects(
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.setLimit(5).build(), context), "limited"
Iterables.limit(expectedResults, 5),
mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build()), context),
"limited"
);
}
@ -3600,7 +3626,7 @@ public class GroupByQueryRunnerTest
);
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
}
@Test
@ -3933,7 +3959,7 @@ public class GroupByQueryRunnerTest
);
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery), context), "merged");
}
@Test
@ -4049,7 +4075,7 @@ public class GroupByQueryRunnerTest
factory.getToolchest().mergeResults(
factory.getToolchest().preMergeQueryDecoration(mergedRunner)
)
).run(fullQuery, context),
).run(QueryPlus.wrap(fullQuery), context),
"merged"
);
@ -4065,7 +4091,7 @@ public class GroupByQueryRunnerTest
factory.getToolchest().mergeResults(
factory.getToolchest().preMergeQueryDecoration(mergedRunner)
)
).run(fullQuery, context),
).run(QueryPlus.wrap(fullQuery), context),
"merged"
);
}
@ -4094,7 +4120,7 @@ public class GroupByQueryRunnerTest
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
Map<String, Object> context = Maps.newHashMap();
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
}
@Test
@ -4153,7 +4179,7 @@ public class GroupByQueryRunnerTest
Map<String, Object> context = Maps.newHashMap();
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query), context), "no-limit");
}
// A subquery identical to the query should yield identical results
@ -6623,7 +6649,11 @@ public class GroupByQueryRunnerTest
)
);
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), "");
TestHelper.assertExpectedObjects(
bySegmentResults,
theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()),
""
);
exec.shutdownNow();
}
@ -6698,7 +6728,7 @@ public class GroupByQueryRunnerTest
)
);
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), "");
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), "");
exec.shutdownNow();
}
@ -6772,7 +6802,7 @@ public class GroupByQueryRunnerTest
)
);
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), "");
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), "");
exec.shutdownNow();
}
@ -7202,7 +7232,7 @@ public class GroupByQueryRunnerTest
)
);
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(fullQuery, Maps.newHashMap()), "");
TestHelper.assertExpectedObjects(bySegmentResults, theRunner.run(QueryPlus.wrap(fullQuery), Maps.newHashMap()), "");
exec.shutdownNow();
}
@ -8687,33 +8717,27 @@ public class GroupByQueryRunnerTest
new QueryRunner<Row>()
{
@Override
public Sequence<Row> run(
Query<Row> query, Map<String, Object> responseContext
)
public Sequence<Row> run(QueryPlus<Row> queryPlus, Map<String, Object> responseContext)
{
// simulate two daily segments
final Query query1 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))
);
final Query query2 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
);
return factory.getToolchest().mergeResults(
new QueryRunner<Row>()
{
@Override
public Sequence<Row> run(Query<Row> query, Map<String, Object> responseContext)
{
return new MergeSequence(
query.getResultOrdering(),
(queryPlus3, responseContext1) -> new MergeSequence<>(
queryPlus3.getQuery().getResultOrdering(),
Sequences.simple(
Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext))
Arrays.asList(
runner.run(queryPlus1, responseContext1),
runner.run(queryPlus2, responseContext1)
)
);
}
}
).run(query, responseContext);
)
)
).run(queryPlus, responseContext);
}
}
);
@ -8726,7 +8750,11 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 6L, "idx", 4420L)
);
TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged");
TestHelper.assertExpectedObjects(
allGranExpectedResults,
mergedRunner.run(QueryPlus.wrap(allGranQuery), context),
"merged"
);
}
@Test
@ -8772,32 +8800,28 @@ public class GroupByQueryRunnerTest
{
@Override
public Sequence<Row> run(
Query<Row> query, Map<String, Object> responseContext
QueryPlus<Row> queryPlus, Map<String, Object> responseContext
)
{
// simulate two daily segments
final Query query1 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))
);
final Query query2 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
);
return factory.getToolchest().mergeResults(
new QueryRunner<Row>()
{
@Override
public Sequence<Row> run(Query<Row> query, Map<String, Object> responseContext)
{
return new MergeSequence(
query.getResultOrdering(),
(queryPlus3, responseContext1) -> new MergeSequence<>(
queryPlus3.getQuery().getResultOrdering(),
Sequences.simple(
Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext))
Arrays.asList(
runner.run(queryPlus1, responseContext1),
runner.run(queryPlus2, responseContext1)
)
);
}
}
).run(query, responseContext);
)
)
).run(queryPlus, responseContext);
}
}
);
@ -8811,7 +8835,10 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L)
);
Iterable<Row> results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.<Row>newArrayList());
Iterable<Row> results = Sequences.toList(
mergedRunner.run(QueryPlus.wrap(allGranQuery), context),
new ArrayList<Row>()
);
TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged");
}
@ -8866,32 +8893,28 @@ public class GroupByQueryRunnerTest
{
@Override
public Sequence<Row> run(
Query<Row> query, Map<String, Object> responseContext
QueryPlus<Row> queryPlus, Map<String, Object> responseContext
)
{
// simulate two daily segments
final Query query1 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))
);
final Query query2 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
);
return factory.getToolchest().mergeResults(
new QueryRunner<Row>()
{
@Override
public Sequence<Row> run(Query<Row> query, Map<String, Object> responseContext)
{
return new MergeSequence(
query.getResultOrdering(),
(queryPlus3, responseContext1) -> new MergeSequence<>(
queryPlus3.getQuery().getResultOrdering(),
Sequences.simple(
Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext))
Arrays.asList(
runner.run(queryPlus1, responseContext1),
runner.run(queryPlus2, responseContext1)
)
);
}
}
).run(query, responseContext);
)
)
).run(queryPlus, responseContext);
}
}
);
@ -8905,7 +8928,10 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L)
);
Iterable<Row> results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.<Row>newArrayList());
Iterable<Row> results = Sequences.toList(
mergedRunner.run(QueryPlus.wrap(allGranQuery), context),
Lists.<Row>newArrayList()
);
TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged");
}
@ -8964,32 +8990,28 @@ public class GroupByQueryRunnerTest
{
@Override
public Sequence<Row> run(
Query<Row> query, Map<String, Object> responseContext
QueryPlus<Row> queryPlus, Map<String, Object> responseContext
)
{
// simulate two daily segments
final Query query1 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))
);
final Query query2 = query.withQuerySegmentSpec(
final QueryPlus<Row> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
);
return factory.getToolchest().mergeResults(
new QueryRunner<Row>()
{
@Override
public Sequence<Row> run(Query<Row> query, Map<String, Object> responseContext)
{
return new MergeSequence(
query.getResultOrdering(),
(queryPlus3, responseContext1) -> new MergeSequence<>(
queryPlus3.getQuery().getResultOrdering(),
Sequences.simple(
Arrays.asList(runner.run(query1, responseContext), runner.run(query2, responseContext))
Arrays.asList(
runner.run(queryPlus1, responseContext1),
runner.run(queryPlus2, responseContext1)
)
);
}
}
).run(query, responseContext);
)
)
).run(queryPlus, responseContext);
}
}
);
@ -9003,7 +9025,10 @@ public class GroupByQueryRunnerTest
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L)
);
Iterable<Row> results = Sequences.toList(mergedRunner.run(allGranQuery, context), Lists.<Row>newArrayList());
Iterable<Row> results = Sequences.toList(
mergedRunner.run(QueryPlus.wrap(allGranQuery), context),
Lists.<Row>newArrayList()
);
TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged");
}
@ -9281,13 +9306,8 @@ public class GroupByQueryRunnerTest
ChainedExecutionQueryRunner ceqr = new ChainedExecutionQueryRunner(
MoreExecutors.sameThreadExecutor(),
new QueryWatcher()
{
@Override
public void registerQuery(Query query, ListenableFuture future)
{
(query1, future) -> {
return;
}
},
ImmutableList.<QueryRunner<Row>>of(runner, runner)
);

View File

@ -28,6 +28,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
@ -51,7 +52,7 @@ public class GroupByQueryRunnerTestHelper
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}

View File

@ -90,9 +90,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
toolChest
);
return Sequences.map(
newRunner.run(
GroupByQuery.builder()
GroupByQuery newQuery = GroupByQuery
.builder()
.setDataSource(tsQuery.getDataSource())
.setQuerySegmentSpec(tsQuery.getQuerySegmentSpec())
.setGranularity(tsQuery.getGranularity())
@ -101,9 +100,10 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
.setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs())
.setVirtualColumns(tsQuery.getVirtualColumns())
.setContext(tsQuery.getContext())
.build(),
responseContext
),
.build();
return Sequences.map(
newRunner.run(queryPlus.withQuery(newQuery), responseContext),
new Function<Row, Result<TimeseriesResultValue>>()
{
@Override
@ -158,7 +158,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
DateTime expectedLast = new DateTime("2011-04-15");
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Result<TimeseriesResultValue> result = results.iterator().next();

View File

@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
import io.druid.data.input.impl.DimensionSchema;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.LegacyDataSource;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -194,6 +195,6 @@ public class SegmentAnalyzerTest
new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null, analyses, false, false
);
HashMap<String, Object> context = new HashMap<String, Object>();
return Sequences.toList(query.run(runner, context), Lists.<SegmentAnalysis>newArrayList());
return Sequences.toList(runner.run(QueryPlus.wrap(query), context), Lists.<SegmentAnalysis>newArrayList());
}
}

View File

@ -36,6 +36,7 @@ import io.druid.query.BySegmentResultValueClass;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -261,7 +262,7 @@ public class SegmentMetadataQueryTest
public void testSegmentMetadataQuery()
{
List<SegmentAnalysis> results = Sequences.toList(
runner1.run(testQuery, Maps.newHashMap()),
runner1.run(QueryPlus.wrap(testQuery), Maps.newHashMap()),
Lists.<SegmentAnalysis>newArrayList()
);
@ -320,18 +321,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement", "placementish")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.ROLLUP)
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -389,18 +389,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement", "placementish")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.CARDINALITY)
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -458,18 +457,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement", "quality_uniques")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.CARDINALITY)
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -580,7 +578,7 @@ public class SegmentMetadataQueryTest
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(query, Maps.newHashMap()),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -628,18 +626,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
.analysisTypes()
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -691,18 +688,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.AGGREGATORS)
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -750,18 +746,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.TIMESTAMPSPEC)
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -809,18 +804,17 @@ public class SegmentMetadataQueryTest
toolChest
);
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(
Druids.newSegmentMetadataQueryBuilder()
SegmentMetadataQuery query = Druids
.newSegmentMetadataQueryBuilder()
.dataSource("testing")
.intervals("2013/2014")
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
.analysisTypes(SegmentMetadataQuery.AnalysisType.QUERYGRANULARITY)
.merge(true)
.build(),
Maps.newHashMap()
),
.build();
TestHelper.assertExpectedObjects(
ImmutableList.of(mergedSegmentAnalysis),
myRunner.run(QueryPlus.wrap(query), Maps.newHashMap()),
"failed SegmentMetadata merging query"
);
exec.shutdownNow();
@ -858,7 +852,7 @@ public class SegmentMetadataQueryTest
TestHelper.assertExpectedObjects(
ImmutableList.of(bySegmentResult, bySegmentResult),
myRunner.run(
testQuery.withOverriddenContext(ImmutableMap.<String, Object>of("bySegment", true)),
QueryPlus.wrap(testQuery.withOverriddenContext(ImmutableMap.<String, Object>of("bySegment", true))),
Maps.newHashMap()
),
"failed SegmentMetadata bySegment query"

View File

@ -25,6 +25,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -128,7 +129,10 @@ public class SegmentMetadataUnionQueryTest
SegmentMetadataQuery.AnalysisType.MINMAX
)
.build();
List result = Sequences.toList(runner.run(query, Maps.newHashMap()), Lists.<SegmentAnalysis>newArrayList());
List result = Sequences.toList(
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<SegmentAnalysis>newArrayList()
);
TestHelper.assertExpectedObjects(ImmutableList.of(expected), result, "failed SegmentMetadata union query");
}

View File

@ -815,7 +815,7 @@ public class SearchQueryRunnerTest
private void checkSearchQuery(Query searchQuery, QueryRunner runner, List<SearchHit> expectedResults)
{
Iterable<Result<SearchResultValue>> results = Sequences.toList(
runner.run(searchQuery, ImmutableMap.of()),
runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()),
Lists.<Result<SearchResultValue>>newArrayList()
);
List<SearchHit> copy = Lists.newLinkedList(expectedResults);

View File

@ -27,6 +27,7 @@ import com.google.common.io.CharSource;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.Result;
import io.druid.query.search.search.AutoStrategy;
@ -243,7 +244,7 @@ public class SearchQueryRunnerWithCaseTest
{
HashMap<String, List> context = new HashMap<>();
Iterable<Result<SearchResultValue>> results = Sequences.toList(
runner.run(searchQuery, context),
runner.run(QueryPlus.<Result<SearchResultValue>>wrap(searchQuery), context),
Lists.<Result<SearchResultValue>>newArrayList()
);

View File

@ -29,6 +29,7 @@ import io.druid.jackson.DefaultObjectMapper;
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.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -248,7 +249,7 @@ public class MultiSegmentSelectQueryTest
{
for (int[] expected : expectedOffsets) {
List<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, ImmutableMap.of()),
runner.run(QueryPlus.wrap(query), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
Assert.assertEquals(1, results.size());
@ -295,7 +296,7 @@ public class MultiSegmentSelectQueryTest
{
for (int[] expected : expectedOffsets) {
List<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, ImmutableMap.of()),
runner.run(QueryPlus.wrap(query), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
Assert.assertEquals(2, results.size());
@ -340,14 +341,17 @@ public class MultiSegmentSelectQueryTest
QueryRunner unionQueryRunner = new UnionQueryRunner(runner);
List<Result<SelectResultValue>> results = Sequences.toList(
unionQueryRunner.run(query, ImmutableMap.of()),
unionQueryRunner.run(QueryPlus.wrap(query), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
Map<String, Integer> pagingIdentifiers = results.get(0).getValue().getPagingIdentifiers();
query = query.withPagingSpec(toNextCursor(PagingSpec.merge(Arrays.asList(pagingIdentifiers)), query, 3));
Sequences.toList(unionQueryRunner.run(query, ImmutableMap.of()), Lists.<Result<SelectResultValue>>newArrayList());
Sequences.toList(
unionQueryRunner.run(QueryPlus.wrap(query), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
}
private PagingSpec toNextCursor(Map<String, Integer> merged, SelectQuery query, int threshold)

View File

@ -33,6 +33,7 @@ import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Sequences;
import io.druid.js.JavaScriptConfig;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -169,7 +170,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -213,7 +214,7 @@ public class SelectQueryRunnerTest
SelectQuery query = newTestQuery().intervals(I_0112_0114).build();
for (int offset : expected) {
List<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, ImmutableMap.of()),
runner.run(QueryPlus.wrap(query), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -230,7 +231,7 @@ public class SelectQueryRunnerTest
query = newTestQuery().intervals(I_0112_0114).build();
for (int offset : expected) {
List<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, ImmutableMap.of()),
runner.run(QueryPlus.wrap(query), ImmutableMap.of()),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -275,7 +276,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -385,7 +386,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -424,7 +425,7 @@ public class SelectQueryRunnerTest
.build();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -462,7 +463,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -536,7 +537,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -584,13 +585,13 @@ public class SelectQueryRunnerTest
.build();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<Result<SelectResultValue>>newArrayList()
);
Iterable<Result<SelectResultValue>> resultsOptimize = Sequences.toList(
toolChest
.postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)))
.run(query, Maps.<String, Object>newHashMap()), Lists.<Result<SelectResultValue>>newArrayList()
.run(QueryPlus.wrap(query), Maps.newHashMap()), Lists.<Result<SelectResultValue>>newArrayList()
);
final List<List<Map<String, Object>>> events = toEvents(
@ -642,7 +643,7 @@ public class SelectQueryRunnerTest
.build();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -690,7 +691,7 @@ public class SelectQueryRunnerTest
.build();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, Maps.newHashMap()),
runner.run(QueryPlus.wrap(query), Maps.newHashMap()),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -730,7 +731,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);
@ -848,7 +849,7 @@ public class SelectQueryRunnerTest
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SelectResultValue>>newArrayList()
);

View File

@ -107,13 +107,13 @@ public class SpecificSegmentQueryRunnerTest
)
)
.build();
Sequence results = queryRunner.run(query, responseContext);
Sequence results = queryRunner.run(QueryPlus.wrap(query), responseContext);
Sequences.toList(results, Lists.newArrayList());
validate(mapper, descriptor, responseContext);
// from toYielder
responseContext = Maps.newHashMap();
results = queryRunner.run(query, responseContext);
results = queryRunner.run(QueryPlus.wrap(query), responseContext);
results.toYielder(
null, new YieldingAccumulator()
{
@ -184,10 +184,7 @@ public class SpecificSegmentQueryRunnerTest
)
)
.build();
Sequence results = queryRunner.run(
query,
responseContext
);
Sequence results = queryRunner.run(QueryPlus.wrap(query), responseContext);
List<Result<TimeseriesResultValue>> res = Sequences.toList(
results,
Lists.<Result<TimeseriesResultValue>>newArrayList()

View File

@ -27,6 +27,7 @@ import com.google.common.io.CharSource;
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.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -175,7 +176,7 @@ public class TimeBoundaryQueryRunnerTest
Assert.assertTrue(timeBoundaryQuery.hasFilters());
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
customRunner.run(timeBoundaryQuery, context),
customRunner.run(QueryPlus.wrap(timeBoundaryQuery), context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
);
@ -201,7 +202,7 @@ public class TimeBoundaryQueryRunnerTest
Assert.assertTrue(timeBoundaryQuery.hasFilters());
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
customRunner.run(timeBoundaryQuery, context),
customRunner.run(QueryPlus.wrap(timeBoundaryQuery), context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
);
@ -218,7 +219,7 @@ public class TimeBoundaryQueryRunnerTest
Assert.assertFalse(timeBoundaryQuery.hasFilters());
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
runner.run(timeBoundaryQuery, context),
runner.run(QueryPlus.wrap(timeBoundaryQuery), context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
);
TimeBoundaryResultValue val = results.iterator().next().getValue();
@ -240,7 +241,7 @@ public class TimeBoundaryQueryRunnerTest
Map<String, Object> context = new MapMaker().makeMap();
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
runner.run(timeBoundaryQuery, context),
runner.run(QueryPlus.wrap(timeBoundaryQuery), context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
);
TimeBoundaryResultValue val = results.iterator().next().getValue();
@ -262,7 +263,7 @@ public class TimeBoundaryQueryRunnerTest
Map<String, Object> context = new MapMaker().makeMap();
context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList());
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
runner.run(timeBoundaryQuery, context),
runner.run(QueryPlus.wrap(timeBoundaryQuery), context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
);
TimeBoundaryResultValue val = results.iterator().next().getValue();

View File

@ -122,7 +122,7 @@ public class TimeSeriesUnionQueryRunnerTest
);
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -228,7 +228,7 @@ public class TimeSeriesUnionQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
mergingrunner.run(query, Maps.<String, Object>newHashMap()),
mergingrunner.run(QueryPlus.wrap(query), Maps.<String, Object>newHashMap()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);

View File

@ -28,6 +28,7 @@ import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -138,7 +139,7 @@ public class TimeseriesQueryRunnerBonusTest
.build();
HashMap<String, Object> context = new HashMap<String, Object>();
return Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
}

View File

@ -31,6 +31,7 @@ import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.QueryToolChest;
@ -70,6 +71,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -156,7 +158,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -177,12 +179,13 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -248,7 +251,7 @@ public class TimeseriesQueryRunnerTest
.build();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -290,7 +293,7 @@ public class TimeseriesQueryRunnerTest
DateTime expectedLast = new DateTime("2011-04-15");
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Result<TimeseriesResultValue> result = results.iterator().next();
@ -338,7 +341,7 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.last;
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -403,7 +406,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -451,7 +454,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -499,7 +502,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -513,7 +516,7 @@ public class TimeseriesQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(new PeriodGranularity(new Period("P1M"), null, null))
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z"
)
@ -532,7 +535,7 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.build();
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults1 = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -542,7 +545,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults1, results1);
@ -551,7 +554,7 @@ public class TimeseriesQueryRunnerTest
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity("DAY")
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z"
)
@ -569,7 +572,7 @@ public class TimeseriesQueryRunnerTest
)
.build();
List<Result<TimeseriesResultValue>> expectedResults2 = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults2 = Collections.singletonList(
new Result<>(
new DateTime("2011-04-02"),
new TimeseriesResultValue(
@ -579,7 +582,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
runner.run(query2, CONTEXT),
runner.run(QueryPlus.wrap(query2), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults2, results2);
@ -599,7 +602,7 @@ public class TimeseriesQueryRunnerTest
)
)
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-01-12T00:00:00.000-08:00/2011-01-20T00:00:00.000-08:00"
)
@ -633,7 +636,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults1, results1);
@ -647,7 +650,7 @@ public class TimeseriesQueryRunnerTest
.filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market")
.granularity(Granularities.HOUR)
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-04-14T00:00:00.000Z/2011-05-01T00:00:00.000Z"
)
@ -680,7 +683,7 @@ public class TimeseriesQueryRunnerTest
List<Result<TimeseriesResultValue>> expectedResults1 = Lists.newArrayList(
Iterables.concat(
Arrays.asList(
Collections.singletonList(
new Result<>(
new DateTime("2011-04-14T00"),
new TimeseriesResultValue(
@ -689,7 +692,7 @@ public class TimeseriesQueryRunnerTest
)
),
lotsOfZeroes,
Arrays.asList(
Collections.singletonList(
new Result<>(
new DateTime("2011-04-15T00"),
new TimeseriesResultValue(
@ -701,7 +704,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults1, results1);
@ -721,7 +724,7 @@ public class TimeseriesQueryRunnerTest
)
)
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-04-15T00:00:00.000Z/2012"
)
@ -739,7 +742,7 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.build();
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults1 = Collections.singletonList(
new Result<>(
new DateTime("2011-04-14T23:01Z"),
new TimeseriesResultValue(
@ -749,7 +752,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults1, results1);
@ -763,7 +766,7 @@ public class TimeseriesQueryRunnerTest
.filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market")
.granularity(new PeriodGranularity(new Period("P1M"), null, null))
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z"
)
@ -782,7 +785,7 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.build();
List<Result<TimeseriesResultValue>> expectedResults1 = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults1 = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -791,7 +794,7 @@ public class TimeseriesQueryRunnerTest
)
);
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
runner.run(query1, CONTEXT),
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults1, results1);
@ -801,7 +804,7 @@ public class TimeseriesQueryRunnerTest
.filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market")
.granularity("DAY")
.intervals(
Arrays.asList(
Collections.singletonList(
new Interval(
"2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z"
)
@ -819,7 +822,7 @@ public class TimeseriesQueryRunnerTest
)
.build();
List<Result<TimeseriesResultValue>> expectedResults2 = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults2 = Collections.singletonList(
new Result<>(
new DateTime("2011-04-02"),
new TimeseriesResultValue(
@ -829,7 +832,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
runner.run(query2, CONTEXT),
runner.run(QueryPlus.wrap(query2), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults2, results2);
@ -843,7 +846,7 @@ public class TimeseriesQueryRunnerTest
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(
new MultipleIntervalSegmentSpec(
Arrays.asList(
Collections.singletonList(
new Interval(
"2015-01-01/2015-01-10"
)
@ -862,10 +865,10 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.emptyList();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -886,7 +889,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -916,7 +920,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -937,7 +941,7 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build();
@ -967,7 +971,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -988,7 +992,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1018,7 +1023,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1039,7 +1044,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1069,7 +1075,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1090,7 +1096,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1120,7 +1127,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1149,7 +1156,8 @@ public class TimeseriesQueryRunnerTest
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1179,7 +1187,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1208,7 +1216,8 @@ public class TimeseriesQueryRunnerTest
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1238,7 +1247,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1267,7 +1276,8 @@ public class TimeseriesQueryRunnerTest
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonDoubleAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1297,7 +1307,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1324,7 +1334,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1354,7 +1365,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1387,7 +1398,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1417,7 +1429,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1450,7 +1462,8 @@ public class TimeseriesQueryRunnerTest
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1480,7 +1493,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1495,7 +1508,8 @@ public class TimeseriesQueryRunnerTest
.filters("bobby", "billy")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1525,7 +1539,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1540,15 +1554,16 @@ public class TimeseriesQueryRunnerTest
.filters("bobby", "billy")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.context(ImmutableMap.<String, Object>of("skipEmptyBuckets", "true"))
.descending(descending)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList();
List<Result<TimeseriesResultValue>> expectedResults = Collections.emptyList();
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, new HashMap<String, Object>()),
runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1563,7 +1578,8 @@ public class TimeseriesQueryRunnerTest
.filters("bobby", null)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1593,7 +1609,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, new HashMap<String, Object>()),
runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1608,7 +1624,8 @@ public class TimeseriesQueryRunnerTest
.filters(new NotDimFilter(new SelectorDimFilter("bobby", "sally", null)))
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1638,7 +1655,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, new HashMap<String, Object>()),
runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1653,7 +1670,8 @@ public class TimeseriesQueryRunnerTest
.filters(QueryRunnerTestHelper.marketDimension, "billy")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1683,7 +1701,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1712,7 +1730,8 @@ public class TimeseriesQueryRunnerTest
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
@ -1742,7 +1761,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, results);
@ -1783,7 +1802,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, actualResults);
@ -1825,7 +1844,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
assertExpectedResults(expectedResults, actualResults);
@ -1929,7 +1948,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
if (descending) {
@ -1942,32 +1961,32 @@ public class TimeseriesQueryRunnerTest
@Test
public void testTimeseriesWithMultiValueDimFilter1()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.placementishDimension, "preferred")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(
Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query1 = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build(),
CONTEXT
),
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -1976,33 +1995,33 @@ public class TimeseriesQueryRunnerTest
@Test
public void testTimeseriesWithMultiValueDimFilter2()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.placementishDimension, "a")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(
Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query1 = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.qualityDimension, "automotive")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build(),
CONTEXT
),
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(QueryPlus.wrap(query1), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -2025,13 +2044,14 @@ public class TimeseriesQueryRunnerTest
)
)
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build();
@ -2050,23 +2070,22 @@ public class TimeseriesQueryRunnerTest
)
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(
Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query2 = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter2)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build(),
CONTEXT
),
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(QueryPlus.wrap(query2), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -2088,13 +2107,14 @@ public class TimeseriesQueryRunnerTest
)
)
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build();
@ -2116,23 +2136,22 @@ public class TimeseriesQueryRunnerTest
)
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(
Druids.newTimeseriesQueryBuilder()
TimeseriesQuery query2 = Druids
.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter2)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(aggregatorFactoryList)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant))
.descending(descending)
.build(),
CONTEXT
),
.build();
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
runner.run(QueryPlus.wrap(query2), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults);
@ -2161,15 +2180,16 @@ public class TimeseriesQueryRunnerTest
)
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -2210,16 +2230,17 @@ public class TimeseriesQueryRunnerTest
)
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -2260,16 +2281,17 @@ public class TimeseriesQueryRunnerTest
)
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -2312,15 +2334,16 @@ public class TimeseriesQueryRunnerTest
)
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -2363,15 +2386,16 @@ public class TimeseriesQueryRunnerTest
)
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.descending(descending)
.build();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -2406,7 +2430,7 @@ public class TimeseriesQueryRunnerTest
.descending(descending)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>(
new DateTime("2011-04-01"),
new TimeseriesResultValue(
@ -2423,7 +2447,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
@ -2480,7 +2504,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
@ -2509,7 +2534,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, results);
@ -2537,7 +2562,8 @@ public class TimeseriesQueryRunnerTest
QueryRunnerTestHelper.qualityUniques
)
)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
.postAggregators(Collections.<PostAggregator>singletonList(QueryRunnerTestHelper
.addRowsIndexConstant))
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
@ -2566,7 +2592,7 @@ public class TimeseriesQueryRunnerTest
);
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, CONTEXT),
runner.run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, results);
@ -2577,7 +2603,7 @@ public class TimeseriesQueryRunnerTest
QueryRunner<Result<TimeseriesResultValue>> optimizedRunner = toolChest.postMergeQueryDecoration(
toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)));
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
new FinalizeResultsQueryRunner(optimizedRunner, toolChest).run(query, CONTEXT),
new FinalizeResultsQueryRunner(optimizedRunner, toolChest).run(QueryPlus.wrap(query), CONTEXT),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, results2);

View File

@ -215,19 +215,21 @@ public class TopNQueryQueryToolChestTest
TopNQuery query1 = builder.threshold(10).context(null).build();
MockQueryRunner mockRunner = new MockQueryRunner(runner);
new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config)
.run(query1, ImmutableMap.<String, Object>of());
new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config).run(
QueryPlus.wrap(query1),
ImmutableMap.<String, Object>of()
);
Assert.assertEquals(1000, mockRunner.query.getThreshold());
TopNQuery query2 = builder.threshold(10).context(context).build();
new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config)
.run(query2, ImmutableMap.<String, Object>of());
.run(QueryPlus.wrap(query2), ImmutableMap.<String, Object>of());
Assert.assertEquals(500, mockRunner.query.getThreshold());
TopNQuery query3 = builder.threshold(2000).context(context).build();
new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config)
.run(query3, ImmutableMap.<String, Object>of());
.run(QueryPlus.wrap(query3), ImmutableMap.<String, Object>of());
Assert.assertEquals(2000, mockRunner.query.getThreshold());
}
@ -248,7 +250,7 @@ public class TopNQueryQueryToolChestTest
)
{
this.query = (TopNQuery) queryPlus.getQuery();
return query.run(runner, responseContext);
return runner.run(queryPlus, responseContext);
}
}
}

View File

@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.collections.StupidPool;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -134,7 +135,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testmMapped()
{
testCaseMap.get(TestCases.mMappedTestIndex).run(query, context);
testCaseMap.get(TestCases.mMappedTestIndex).run(QueryPlus.wrap(query), context);
}
@Ignore
@ -142,7 +143,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testrtIndex()
{
testCaseMap.get(TestCases.rtIndex).run(query, context);
testCaseMap.get(TestCases.rtIndex).run(QueryPlus.wrap(query), context);
}
@Ignore
@ -150,7 +151,7 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testMerged()
{
testCaseMap.get(TestCases.mergedRealtimeIndex).run(query, context);
testCaseMap.get(TestCases.mergedRealtimeIndex).run(QueryPlus.wrap(query), context);
}
@Ignore
@ -158,6 +159,6 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark
@Test
public void testOffHeap()
{
testCaseMap.get(TestCases.rtIndexOffheap).run(query, context);
testCaseMap.get(TestCases.rtIndexOffheap).run(QueryPlus.wrap(query), context);
}
}

View File

@ -44,6 +44,7 @@ import io.druid.query.BySegmentResultValue;
import io.druid.query.BySegmentResultValueClass;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -267,7 +268,7 @@ public class TopNQueryRunnerTest
chest.mergeResults(runner),
chest
);
return mergeRunner.run(query, context);
return mergeRunner.run(QueryPlus.wrap(query), context);
}
@Test
@ -4048,7 +4049,7 @@ public class TopNQueryRunnerTest
)
)
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()));
}
@Test
@ -4087,7 +4088,7 @@ public class TopNQueryRunnerTest
)
)
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, new HashMap<String, Object>()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()));
}
@ -4210,7 +4211,7 @@ public class TopNQueryRunnerTest
chest.mergeResults(chest.preMergeQueryDecoration(runner)),
chest
);
return Runner.run(query, context);
return Runner.run(QueryPlus.wrap(query), context);
}
@Test

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.collections.StupidPool;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -180,7 +181,7 @@ public class TopNUnionQueryTest
)
);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}

View File

@ -26,6 +26,7 @@ import io.druid.java.util.common.Pair;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -182,7 +183,7 @@ public class AppendTest
.build();
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -207,7 +208,7 @@ public class AppendTest
.build();
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -232,7 +233,7 @@ public class AppendTest
TimeseriesQuery query = makeTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -257,7 +258,7 @@ public class AppendTest
TimeseriesQuery query = makeTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -282,7 +283,7 @@ public class AppendTest
TimeseriesQuery query = makeFilteredTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -307,7 +308,7 @@ public class AppendTest
TimeseriesQuery query = makeFilteredTimeseriesQuery();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -353,7 +354,7 @@ public class AppendTest
TopNQuery query = makeTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -399,7 +400,7 @@ public class AppendTest
TopNQuery query = makeTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -427,7 +428,7 @@ public class AppendTest
TopNQuery query = makeFilteredTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -445,7 +446,7 @@ public class AppendTest
TopNQuery query = makeFilteredTopNQuery();
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -468,7 +469,7 @@ public class AppendTest
SearchQuery query = makeSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -490,7 +491,7 @@ public class AppendTest
SearchQuery query = makeSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -511,7 +512,7 @@ public class AppendTest
SearchQuery query = makeFilteredSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -533,7 +534,7 @@ public class AppendTest
SearchQuery query = makeFilteredSearchQuery();
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -575,7 +576,7 @@ public class AppendTest
.build();
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
private TimeseriesQuery makeTimeseriesQuery()

View File

@ -31,6 +31,7 @@ import io.druid.data.input.impl.SpatialDimensionSchema;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -578,7 +579,7 @@ public class IndexMergerV9WithSpatialIndexTest
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -632,7 +633,7 @@ public class IndexMergerV9WithSpatialIndexTest
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -721,7 +722,7 @@ public class IndexMergerV9WithSpatialIndexTest
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -28,6 +28,7 @@ import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -1452,7 +1453,7 @@ public class SchemalessTestFullTest
failMsg += " timeseries ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1486,7 +1487,7 @@ public class SchemalessTestFullTest
failMsg += " filtered timeseries ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeseriesResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1519,7 +1520,7 @@ public class SchemalessTestFullTest
failMsg += " topN ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TopNResultValue>>newArrayList()
);
@ -1553,7 +1554,7 @@ public class SchemalessTestFullTest
failMsg += " filtered topN ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<TopNResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TopNResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1571,7 +1572,7 @@ public class SchemalessTestFullTest
failMsg += " search ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SearchResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1590,7 +1591,7 @@ public class SchemalessTestFullTest
failMsg += " filtered search ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<SearchResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<SearchResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);
@ -1609,7 +1610,7 @@ public class SchemalessTestFullTest
failMsg += " timeBoundary ";
HashMap<String, Object> context = new HashMap<>();
Iterable<Result<TimeBoundaryResultValue>> actualResults = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
);
TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg);

View File

@ -25,6 +25,7 @@ import com.google.common.collect.Lists;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.query.Druids;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.Result;
import io.druid.query.TestQueryRunners;
@ -167,7 +168,7 @@ public class SchemalessTestSimpleTest
);
QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@ -241,7 +242,7 @@ public class SchemalessTestSimpleTest
QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -270,7 +271,7 @@ public class SchemalessTestSimpleTest
QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
@Test
@ -296,6 +297,6 @@ public class SchemalessTestSimpleTest
QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
}

View File

@ -41,6 +41,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -484,7 +485,7 @@ public class IncrementalIndexTest
List<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, new HashMap<String, Object>()),
runner.run(QueryPlus.wrap(query), new HashMap<String, Object>()),
new LinkedList<Result<TimeseriesResultValue>>()
);
Result<TimeseriesResultValue> result = Iterables.getOnlyElement(results);
@ -642,7 +643,7 @@ public class IncrementalIndexTest
factory.getToolchest()
);
Map<String, Object> context = new HashMap<String, Object>();
Sequence<Result<TimeseriesResultValue>> sequence = runner.run(query, context);
Sequence<Result<TimeseriesResultValue>> sequence = runner.run(QueryPlus.wrap(query), context);
for (Double result :
sequence.accumulate(
@ -701,7 +702,7 @@ public class IncrementalIndexTest
.build();
Map<String, Object> context = new HashMap<String, Object>();
List<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
new LinkedList<Result<TimeseriesResultValue>>()
);
boolean isRollup = index.isRollup();

View File

@ -32,6 +32,7 @@ import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -512,7 +513,7 @@ public class SpatialFilterBonusTest
factory.getToolchest()
);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -600,7 +601,7 @@ public class SpatialFilterBonusTest
factory.getToolchest()
);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -694,7 +695,7 @@ public class SpatialFilterBonusTest
factory.getToolchest()
);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), context));
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -31,6 +31,7 @@ import io.druid.data.input.impl.SpatialDimensionSchema;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
@ -568,7 +569,7 @@ public class SpatialFilterTest
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -621,7 +622,7 @@ public class SpatialFilterTest
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -709,7 +710,7 @@ public class SpatialFilterTest
factory.getToolchest()
);
TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap()));
TestHelper.assertExpectedResults(expectedResults, runner.run(QueryPlus.wrap(query), Maps.newHashMap()));
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -40,6 +40,7 @@ import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryRunnerTestHelper;
@ -393,7 +394,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
Map<String, Object> context = new HashMap<String, Object>();
for (Result<TimeseriesResultValue> result :
Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
new LinkedList<Result<TimeseriesResultValue>>()
)
) {
@ -429,7 +430,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
.build();
Map<String, Object> context = new HashMap<String, Object>();
List<Result<TimeseriesResultValue>> results = Sequences.toList(
runner.run(query, context),
runner.run(QueryPlus.wrap(query), context),
new LinkedList<Result<TimeseriesResultValue>>()
);
final int expectedVal = elementsPerThread * taskCount;

View File

@ -529,6 +529,10 @@ public class CachingClusteredClientTest
HashMap<String, List> context = new HashMap<String, List>();
TimeseriesQuery query = builder.intervals("2011-01-01/2011-01-10")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeRenamedTimeResults(
new DateTime("2011-01-01"), 50, 5000,
@ -545,13 +549,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), 18, 521,
new DateTime("2011-01-09T01"), 181, 52
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -591,7 +589,7 @@ public class CachingClusteredClientTest
selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), dataSegment);
timeline.add(interval, "v", new SingleElementPartitionChunk<>(selector));
getDefaultQueryRunner().run(query, context);
getDefaultQueryRunner().run(QueryPlus.wrap(query), context);
Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured());
Assert.assertTrue("Cache key below limit", ImmutableList.copyOf(cacheKeyCapture.getValue()).size() <= limit);
@ -605,7 +603,7 @@ public class CachingClusteredClientTest
.once();
EasyMock.replay(cache);
client = makeClient(MoreExecutors.sameThreadExecutor(), cache, 0);
getDefaultQueryRunner().run(query, context);
getDefaultQueryRunner().run(QueryPlus.wrap(query), context);
EasyMock.verify(cache);
EasyMock.verify(dataSegment);
Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured());
@ -651,6 +649,11 @@ public class CachingClusteredClientTest
)
);
TimeseriesQuery query = builder
.intervals("2011-01-05/2011-01-10")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeRenamedTimeResults(
new DateTime("2011-01-05T00"), 85, 102,
@ -664,13 +667,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09T00"), 18, 521,
new DateTime("2011-01-09T02"), 181, 52
),
runner.run(
builder.intervals("2011-01-05/2011-01-10")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
.build(),
Maps.newHashMap()
)
runner.run(QueryPlus.wrap(query), Maps.newHashMap())
);
}
@ -705,6 +702,11 @@ public class CachingClusteredClientTest
)
);
HashMap<String, List> context = new HashMap<String, List>();
TimeseriesQuery query = builder
.intervals("2011-11-04/2011-11-08")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeRenamedTimeResults(
new DateTime("2011-11-04", TIMEZONE), 50, 5000,
@ -712,13 +714,7 @@ public class CachingClusteredClientTest
new DateTime("2011-11-06", TIMEZONE), 23, 85312,
new DateTime("2011-11-07", TIMEZONE), 85, 102
),
runner.run(
builder.intervals("2011-11-04/2011-11-08")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS)
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -844,6 +840,12 @@ public class CachingClusteredClientTest
)
);
HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-01-01/2011-01-10")
.metric("imps")
.aggregators(RENAMED_AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeRenamedTopNResults(
new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998,
@ -859,14 +861,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983,
new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.metric("imps")
.aggregators(RENAMED_AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -905,6 +900,12 @@ public class CachingClusteredClientTest
)
);
HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-11-04/2011-11-08")
.metric("imps")
.aggregators(RENAMED_AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeRenamedTopNResults(
@ -913,14 +914,7 @@ public class CachingClusteredClientTest
new DateTime("2011-11-06", TIMEZONE), "a", 50, 4991, "b", 50, 4990, "c", 50, 4989,
new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986
),
runner.run(
builder.intervals("2011-11-04/2011-11-08")
.metric("imps")
.aggregators(RENAMED_AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -1027,6 +1021,12 @@ public class CachingClusteredClientTest
);
HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-01-01/2011-01-10")
.metric("imps")
.aggregators(RENAMED_AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeRenamedTopNResults(
new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992,
@ -1040,14 +1040,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983,
new DateTime("2011-01-09T01"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.metric("imps")
.aggregators(RENAMED_AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -1101,6 +1094,12 @@ public class CachingClusteredClientTest
);
HashMap<String, List> context = new HashMap<String, List>();
TopNQuery query = builder
.intervals("2011-01-01/2011-01-10")
.metric("avg_imps_per_row_double")
.aggregators(AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build();
TestHelper.assertExpectedResults(
makeTopNResultsWithoutRename(
new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992,
@ -1114,14 +1113,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), "c1", 50, 4985, "b", 50, 4984, "c", 50, 4983,
new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.metric("avg_imps_per_row_double")
.aggregators(AGGS)
.postAggregators(DIFF_ORDER_POST_AGGS)
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -1191,11 +1183,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4,
new DateTime("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.build(),
context
)
runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
);
}
@ -1265,12 +1253,12 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4,
new DateTime("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.build(),
context
)
runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
);
SearchQuery query = builder
.intervals("2011-01-01/2011-01-10")
.dimensions(new DefaultDimensionSpec(TOP_DIM, "new_dim"))
.build();
TestHelper.assertExpectedResults(
makeSearchResults(
"new_dim",
@ -1287,15 +1275,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4,
new DateTime("2011-01-09T01"), "how6", 1, "howdy6", 2, "howwwwww6", 3, "howww6", 4
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.dimensions(new DefaultDimensionSpec(
TOP_DIM,
"new_dim"
))
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -1364,11 +1344,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9),
new DateTime("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.build(),
context
)
runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
);
}
@ -1443,13 +1419,13 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9),
new DateTime("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9)
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.build(),
context
)
runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context)
);
SelectQuery query = builder
.intervals("2011-01-01/2011-01-10")
.dimensionSpecs(Lists.newArrayList(new DefaultDimensionSpec("a", "a2")))
.build();
TestHelper.assertExpectedResults(
makeSelectResults(
dimensions, metrics,
@ -1466,12 +1442,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09"), ImmutableMap.of("a2", "h", "rows", 9),
new DateTime("2011-01-09T01"), ImmutableMap.of("a2", "h", "rows", 9)
),
runner.run(
builder.intervals("2011-01-01/2011-01-10")
.dimensionSpecs(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("a", "a2")))
.build(),
context
)
runner.run(QueryPlus.wrap(query), context)
);
}
@ -1571,11 +1542,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09T01"),
ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector)
),
runner.run(
builder.setInterval("2011-01-05/2011-01-10")
.build(),
context
),
runner.run(QueryPlus.wrap(builder.setInterval("2011-01-05/2011-01-10").build()), context),
""
);
}
@ -2198,11 +2165,9 @@ public class CachingClusteredClientTest
)
),
runner.run(
QueryPlus.wrap(
query.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(
ImmutableList.of(
actualQueryInterval
)
new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval))
)
),
context
@ -3138,14 +3103,15 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09T"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7),
new DateTime("2011-01-09T01"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7)
),
runner.run(
builder.setInterval("2011-01-05/2011-01-10")
.build(),
context
),
runner.run(QueryPlus.wrap(builder.setInterval("2011-01-05/2011-01-10").build()), context),
""
);
GroupByQuery query = builder
.setInterval("2011-01-05/2011-01-10")
.setDimensions(Collections.singletonList(new DefaultDimensionSpec("a", "output2")))
.setAggregatorSpecs(RENAMED_AGGS)
.build();
TestHelper.assertExpectedObjects(
makeGroupByResults(
new DateTime("2011-01-05T"), ImmutableMap.of("output2", "c", "rows", 3, "imps", 3, "impers2", 3),
@ -3159,13 +3125,7 @@ public class CachingClusteredClientTest
new DateTime("2011-01-09T"), ImmutableMap.of("output2", "g", "rows", 7, "imps", 7, "impers2", 7),
new DateTime("2011-01-09T01"), ImmutableMap.of("output2", "g", "rows", 7, "imps", 7, "impers2", 7)
),
runner.run(
builder.setInterval("2011-01-05/2011-01-10")
.setDimensions(Arrays.<DimensionSpec>asList(new DefaultDimensionSpec("a", "output2")))
.setAggregatorSpecs(RENAMED_AGGS)
.build(),
context
),
runner.run(QueryPlus.wrap(query), context),
"renamed aggregators test"
);
}
@ -3204,7 +3164,7 @@ public class CachingClusteredClientTest
Map<String, Object> responseContext = new HashMap<>();
getDefaultQueryRunner().run(query, responseContext);
getDefaultQueryRunner().run(QueryPlus.wrap(query), responseContext);
Assert.assertEquals("Z/eS4rQz5v477iq7Aashr6JPZa0=", responseContext.get("ETag"));
}

View File

@ -300,7 +300,7 @@ public class CachingQueryRunnerTest
);
HashMap<String, Object> context = new HashMap<String, Object>();
Sequence res = runner.run(query, context);
Sequence res = runner.run(QueryPlus.wrap(query), context);
// base sequence is not closed yet
Assert.assertFalse("sequence must not be closed", closable.isClosed());
Assert.assertNull("cache must be empty", cache.get(cacheKey));
@ -386,7 +386,7 @@ public class CachingQueryRunnerTest
);
HashMap<String, Object> context = new HashMap<String, Object>();
List<Result> results = Sequences.toList(runner.run(query, context), new ArrayList());
List<Result> results = Sequences.toList(runner.run(QueryPlus.wrap(query), context), new ArrayList());
Assert.assertEquals(expectedResults.toString(), results.toString());
}

View File

@ -33,12 +33,13 @@ import io.druid.client.selector.ConnectionCountServerSelectorStrategy;
import io.druid.client.selector.HighestPriorityTierSelectorStrategy;
import io.druid.client.selector.QueryableDruidServer;
import io.druid.client.selector.ServerSelector;
import io.druid.java.util.common.StringUtils;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Druids;
import io.druid.query.QueryInterruptedException;
import io.druid.query.QueryPlus;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.ReflectionQueryToolChestWarehouse;
import io.druid.query.Result;
@ -165,22 +166,22 @@ public class DirectDruidClientTest
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
Sequence s1 = client1.run(query, defaultContext);
Sequence s1 = client1.run(QueryPlus.wrap(query), defaultContext);
Assert.assertTrue(capturedRequest.hasCaptured());
Assert.assertEquals(url, capturedRequest.getValue().getUrl());
Assert.assertEquals(HttpMethod.POST, capturedRequest.getValue().getMethod());
Assert.assertEquals(1, client1.getNumOpenConnections());
// simulate read timeout
Sequence s2 = client1.run(query, defaultContext);
Sequence s2 = client1.run(QueryPlus.wrap(query), defaultContext);
Assert.assertEquals(2, client1.getNumOpenConnections());
futureException.setException(new ReadTimeoutException());
Assert.assertEquals(1, client1.getNumOpenConnections());
// subsequent connections should work
Sequence s3 = client1.run(query, defaultContext);
Sequence s4 = client1.run(query, defaultContext);
Sequence s5 = client1.run(query, defaultContext);
Sequence s3 = client1.run(QueryPlus.wrap(query), defaultContext);
Sequence s4 = client1.run(QueryPlus.wrap(query), defaultContext);
Sequence s5 = client1.run(QueryPlus.wrap(query), defaultContext);
Assert.assertTrue(client1.getNumOpenConnections() == 4);
@ -195,8 +196,8 @@ public class DirectDruidClientTest
Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp());
Assert.assertEquals(3, client1.getNumOpenConnections());
client2.run(query, defaultContext);
client2.run(query, defaultContext);
client2.run(QueryPlus.wrap(query), defaultContext);
client2.run(QueryPlus.wrap(query), defaultContext);
Assert.assertTrue(client2.getNumOpenConnections() == 2);
@ -269,7 +270,7 @@ public class DirectDruidClientTest
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled")));
Sequence results = client1.run(query, defaultContext);
Sequence results = client1.run(QueryPlus.wrap(query), defaultContext);
Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod());
Assert.assertEquals(0, client1.getNumOpenConnections());
@ -344,7 +345,7 @@ public class DirectDruidClientTest
StringUtils.toUtf8("{\"error\":\"testing1\",\"errorMessage\":\"testing2\"}")
)
);
Sequence results = client1.run(query, defaultContext);
Sequence results = client1.run(QueryPlus.wrap(query), defaultContext);
QueryInterruptedException actualException = null;
try {

View File

@ -31,6 +31,7 @@ import io.druid.data.input.MapBasedInputRow;
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.QueryPlus;
import io.druid.query.Result;
import io.druid.query.SegmentDescriptor;
import io.druid.query.aggregation.AggregatorFactory;
@ -313,7 +314,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results1 = Lists.newArrayList();
Sequences.toList(query1.run(appenderator, ImmutableMap.<String, Object>of()), results1);
Sequences.toList(QueryPlus.wrap(query1).run(appenderator, ImmutableMap.of()), results1);
Assert.assertEquals(
"query1",
ImmutableList.of(
@ -339,7 +340,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results2 = Lists.newArrayList();
Sequences.toList(query2.run(appenderator, ImmutableMap.<String, Object>of()), results2);
Sequences.toList(QueryPlus.wrap(query2).run(appenderator, ImmutableMap.of()), results2);
Assert.assertEquals(
"query2",
ImmutableList.of(
@ -369,7 +370,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results3 = Lists.newArrayList();
Sequences.toList(query3.run(appenderator, ImmutableMap.<String, Object>of()), results3);
Sequences.toList(QueryPlus.wrap(query3).run(appenderator, ImmutableMap.of()), results3);
Assert.assertEquals(
ImmutableList.of(
new Result<>(
@ -403,7 +404,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results4 = Lists.newArrayList();
Sequences.toList(query4.run(appenderator, ImmutableMap.<String, Object>of()), results4);
Sequences.toList(QueryPlus.wrap(query4).run(appenderator, ImmutableMap.of()), results4);
Assert.assertEquals(
ImmutableList.of(
new Result<>(
@ -459,7 +460,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results1 = Lists.newArrayList();
Sequences.toList(query1.run(appenderator, ImmutableMap.<String, Object>of()), results1);
Sequences.toList(QueryPlus.wrap(query1).run(appenderator, ImmutableMap.of()), results1);
Assert.assertEquals(
"query1",
ImmutableList.of(
@ -495,7 +496,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results2 = Lists.newArrayList();
Sequences.toList(query2.run(appenderator, ImmutableMap.<String, Object>of()), results2);
Sequences.toList(QueryPlus.wrap(query2).run(appenderator, ImmutableMap.of()), results2);
Assert.assertEquals(
"query2",
ImmutableList.of(
@ -536,7 +537,7 @@ public class AppenderatorTest
.build();
final List<Result<TimeseriesResultValue>> results3 = Lists.newArrayList();
Sequences.toList(query3.run(appenderator, ImmutableMap.<String, Object>of()), results3);
Sequences.toList(QueryPlus.wrap(query3).run(appenderator, ImmutableMap.of()), results3);
Assert.assertEquals(
"query2",
ImmutableList.of(

View File

@ -442,7 +442,7 @@ public class ServerManagerTest
public void run()
{
Map<String, Object> context = new HashMap<String, Object>();
Sequence<Result<SearchResultValue>> seq = runner.run(query, context);
Sequence<Result<SearchResultValue>> seq = runner.run(QueryPlus.wrap(query), context);
Sequences.toList(seq, Lists.<Result<SearchResultValue>>newArrayList());
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();