mirror of https://github.com/apache/druid.git
change the signature of run()
This commit is contained in:
parent
f4b1dc032b
commit
8515a11787
|
@ -70,14 +70,14 @@ public abstract class BaseQuery<T> implements Query<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(QuerySegmentWalker walker, Map<String, List> metadata)
|
public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
return run(querySegmentSpec.lookup(this, walker), metadata);
|
return run(querySegmentSpec.lookup(this, walker), context);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Sequence<T> run(QueryRunner<T> runner, Map<String, List> metadata)
|
public Sequence<T> run(QueryRunner<T> runner, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
return runner.run(this, metadata);
|
return runner.run(this, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -52,10 +52,10 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public Sequence<T> run(final Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
if (query.getContextBySegment(false)) {
|
if (query.getContextBySegment(false)) {
|
||||||
final Sequence<T> baseSequence = base.run(query, metadata);
|
final Sequence<T> baseSequence = base.run(query, context);
|
||||||
return new Sequence<T>()
|
return new Sequence<T>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -97,6 +97,6 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
return base.run(query, metadata);
|
return base.run(query, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query;
|
||||||
|
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -38,14 +37,14 @@ public abstract class BySegmentSkippingQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
if (query.getContextBySegment(false)) {
|
if (query.getContextBySegment(false)) {
|
||||||
return baseRunner.run(query, metadata);
|
return baseRunner.run(query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
return doRun(baseRunner, query, metadata);
|
return doRun(baseRunner, query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query, Map<String, List> metadata);
|
protected abstract Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query, Map<String, Object> context);
|
||||||
}
|
}
|
||||||
|
|
|
@ -94,7 +94,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final int priority = query.getContextPriority(0);
|
final int priority = query.getContextPriority(0);
|
||||||
|
|
||||||
|
@ -125,7 +125,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
|
||||||
throw new ISE("Input is null?! How is this possible?!");
|
throw new ISE("Input is null?! How is this possible?!");
|
||||||
}
|
}
|
||||||
|
|
||||||
Sequence<T> result = input.run(query, metadata);
|
Sequence<T> result = input.run(query, context);
|
||||||
if (result == null) {
|
if (result == null) {
|
||||||
throw new ISE("Got a null result! Segments are missing!");
|
throw new ISE("Got a null result! Segments are missing!");
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,7 +39,7 @@ public class ConcatQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
return Sequences.concat(
|
return Sequences.concat(
|
||||||
Sequences.map(
|
Sequences.map(
|
||||||
|
@ -49,7 +49,7 @@ public class ConcatQueryRunner<T> implements QueryRunner<T>
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> apply(final QueryRunner<T> input)
|
public Sequence<T> apply(final QueryRunner<T> input)
|
||||||
{
|
{
|
||||||
return input.run(query, metadata);
|
return input.run(query, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
|
|
|
@ -48,7 +48,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final boolean isBySegment = query.getContextBySegment(false);
|
final boolean isBySegment = query.getContextBySegment(false);
|
||||||
final boolean shouldFinalize = query.getContextFinalize(true);
|
final boolean shouldFinalize = query.getContextFinalize(true);
|
||||||
|
@ -96,7 +96,7 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
|
||||||
|
|
||||||
|
|
||||||
return Sequences.map(
|
return Sequences.map(
|
||||||
baseRunner.run(queryToRun, metadata),
|
baseRunner.run(queryToRun, context),
|
||||||
finalizerFn
|
finalizerFn
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -87,7 +87,7 @@ public class GroupByParallelQueryRunner implements QueryRunner<Row>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Row> run(final Query<Row> queryParam, final Map<String, List> metadata)
|
public Sequence<Row> run(final Query<Row> queryParam, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
|
|
||||||
final GroupByQuery query = (GroupByQuery) queryParam;
|
final GroupByQuery query = (GroupByQuery) queryParam;
|
||||||
|
@ -116,7 +116,7 @@ public class GroupByParallelQueryRunner implements QueryRunner<Row>
|
||||||
public Boolean call() throws Exception
|
public Boolean call() throws Exception
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
input.run(queryParam, metadata).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
input.run(queryParam, context).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
catch (QueryInterruptedException e) {
|
catch (QueryInterruptedException e) {
|
||||||
|
|
|
@ -49,10 +49,10 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
if (period.getMillis() == 0) {
|
if (period.getMillis() == 0) {
|
||||||
return baseRunner.run(query, metadata);
|
return baseRunner.run(query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
return Sequences.concat(
|
return Sequences.concat(
|
||||||
|
@ -76,7 +76,7 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
|
||||||
{
|
{
|
||||||
return baseRunner.run(
|
return baseRunner.run(
|
||||||
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))),
|
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))),
|
||||||
metadata
|
context
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -68,7 +68,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final ServiceMetricEvent.Builder builder = builderFn.apply(query);
|
final ServiceMetricEvent.Builder builder = builderFn.apply(query);
|
||||||
String queryId = query.getId();
|
String queryId = query.getId();
|
||||||
|
@ -86,7 +86,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
|
||||||
|
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
try {
|
try {
|
||||||
retVal = queryRunner.run(query, metadata).accumulate(outType, accumulator);
|
retVal = queryRunner.run(query, context).accumulate(outType, accumulator);
|
||||||
}
|
}
|
||||||
catch (RuntimeException e) {
|
catch (RuntimeException e) {
|
||||||
builder.setUser10("failed");
|
builder.setUser10("failed");
|
||||||
|
@ -116,7 +116,7 @@ public class MetricsEmittingQueryRunner<T> implements QueryRunner<T>
|
||||||
|
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
try {
|
try {
|
||||||
retVal = queryRunner.run(query, metadata).toYielder(initValue, accumulator);
|
retVal = queryRunner.run(query, context).toYielder(initValue, accumulator);
|
||||||
}
|
}
|
||||||
catch (RuntimeException e) {
|
catch (RuntimeException e) {
|
||||||
builder.setUser10("failed");
|
builder.setUser10("failed");
|
||||||
|
|
|
@ -30,7 +30,7 @@ import java.util.Map;
|
||||||
public class NoopQueryRunner<T> implements QueryRunner<T>
|
public class NoopQueryRunner<T> implements QueryRunner<T>
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
return Sequences.empty();
|
return Sequences.empty();
|
||||||
}
|
}
|
||||||
|
|
|
@ -62,9 +62,9 @@ public interface Query<T>
|
||||||
|
|
||||||
public String getType();
|
public String getType();
|
||||||
|
|
||||||
public Sequence<T> run(QuerySegmentWalker walker, Map<String, List> metadata);
|
public Sequence<T> run(QuerySegmentWalker walker, Map<String, Object> context);
|
||||||
|
|
||||||
public Sequence<T> run(QueryRunner<T> runner, Map<String, List> metadata);
|
public Sequence<T> run(QueryRunner<T> runner, Map<String, Object> context);
|
||||||
|
|
||||||
public List<Interval> getIntervals();
|
public List<Interval> getIntervals();
|
||||||
|
|
||||||
|
|
|
@ -21,12 +21,11 @@ package io.druid.query;
|
||||||
|
|
||||||
import com.metamx.common.guava.Sequence;
|
import com.metamx.common.guava.Sequence;
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface QueryRunner<T>
|
public interface QueryRunner<T>
|
||||||
{
|
{
|
||||||
public Sequence<T> run(Query<T> query, Map<String, List> metadata);
|
public Sequence<T> run(Query<T> query, Map<String, Object> context);
|
||||||
}
|
}
|
|
@ -45,11 +45,11 @@ public class ReferenceCountingSegmentQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final Closeable closeable = adapter.increment();
|
final Closeable closeable = adapter.increment();
|
||||||
try {
|
try {
|
||||||
final Sequence<T> baseSequence = factory.createRunner(adapter).run(query, metadata);
|
final Sequence<T> baseSequence = factory.createRunner(adapter).run(query, context);
|
||||||
|
|
||||||
return new ResourceClosingSequence<T>(baseSequence, closeable);
|
return new ResourceClosingSequence<T>(baseSequence, closeable);
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,9 +39,9 @@ public abstract class ResultMergeQueryRunner<T> extends BySegmentSkippingQueryRu
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query, Map<String, List> metadata)
|
public Sequence<T> doRun(QueryRunner<T> baseRunner, Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
return CombiningSequence.create(baseRunner.run(query, metadata), makeOrdering(query), createMergeFn(query));
|
return CombiningSequence.create(baseRunner.run(query, context), makeOrdering(query), createMergeFn(query));
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract Ordering<T> makeOrdering(Query<T> query);
|
protected abstract Ordering<T> makeOrdering(Query<T> query);
|
||||||
|
|
|
@ -42,20 +42,20 @@ public class RetryQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
Sequence<T> returningSeq = baseRunner.run(query, metadata);
|
Sequence<T> returningSeq = baseRunner.run(query, context);
|
||||||
|
|
||||||
for (int i = config.numTries(); i > 0; i--) {
|
for (int i = config.numTries(); i > 0; i--) {
|
||||||
for (int j = metadata.get("missingSegments").size(); j > 0; j--) {
|
for (int j = ((List)context.get("missingSegments")).size(); j > 0; j--) {
|
||||||
QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)metadata.get("missingSegments").remove(0));
|
QuerySegmentSpec segmentSpec = new SpecificSegmentSpec((SegmentDescriptor)((List) context.get("missingSegments")).remove(0));
|
||||||
returningSeq = toolChest.mergeSequences(
|
returningSeq = toolChest.mergeSequences(
|
||||||
Sequences.simple(
|
Sequences.simple(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
returningSeq,
|
returningSeq,
|
||||||
baseRunner.run(
|
baseRunner.run(
|
||||||
query.withQuerySegmentSpec(segmentSpec),
|
query.withQuerySegmentSpec(segmentSpec),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
|
@ -39,13 +39,13 @@ public class SubqueryQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
DataSource dataSource = query.getDataSource();
|
DataSource dataSource = query.getDataSource();
|
||||||
if (dataSource instanceof QueryDataSource) {
|
if (dataSource instanceof QueryDataSource) {
|
||||||
return run((Query<T>) ((QueryDataSource) dataSource).getQuery(), metadata);
|
return run((Query<T>) ((QueryDataSource) dataSource).getQuery(), context);
|
||||||
} else {
|
} else {
|
||||||
return baseRunner.run(query, metadata);
|
return baseRunner.run(query, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,7 +44,7 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
DataSource dataSource = query.getDataSource();
|
DataSource dataSource = query.getDataSource();
|
||||||
if (dataSource instanceof UnionDataSource) {
|
if (dataSource instanceof UnionDataSource) {
|
||||||
|
@ -59,7 +59,7 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
|
||||||
{
|
{
|
||||||
return baseRunner.run(
|
return baseRunner.run(
|
||||||
query.withDataSource(singleSource),
|
query.withDataSource(singleSource),
|
||||||
metadata
|
context
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -67,7 +67,7 @@ public class UnionQueryRunner<T> implements QueryRunner<T>
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
return baseRunner.run(query, metadata);
|
return baseRunner.run(query, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -80,18 +80,18 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
||||||
return new QueryRunner<Row>()
|
return new QueryRunner<Row>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Row> run(Query<Row> input, Map<String, List> metadata)
|
public Sequence<Row> run(Query<Row> input, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
|
if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
|
||||||
return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner, metadata);
|
return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner, context);
|
||||||
} else {
|
} else {
|
||||||
return runner.run(input, metadata);
|
return runner.run(input, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner, Map<String, List> metadata)
|
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
|
|
||||||
Sequence<Row> result;
|
Sequence<Row> result;
|
||||||
|
@ -105,12 +105,12 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
||||||
} catch (ClassCastException e) {
|
} catch (ClassCastException e) {
|
||||||
throw new UnsupportedOperationException("Subqueries must be of type 'group by'");
|
throw new UnsupportedOperationException("Subqueries must be of type 'group by'");
|
||||||
}
|
}
|
||||||
Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner, metadata);
|
Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner, context);
|
||||||
IncrementalIndexStorageAdapter adapter
|
IncrementalIndexStorageAdapter adapter
|
||||||
= new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult));
|
= new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult));
|
||||||
result = engine.process(query, adapter);
|
result = engine.process(query, adapter);
|
||||||
} else {
|
} else {
|
||||||
result = runner.run(query, metadata);
|
result = runner.run(query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
return postAggregate(query, makeIncrementalIndex(query, result));
|
return postAggregate(query, makeIncrementalIndex(query, result));
|
||||||
|
|
|
@ -102,7 +102,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
||||||
return new QueryRunner<Row>()
|
return new QueryRunner<Row>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Row> run(final Query<Row> query, final Map<String, List> metadata)
|
public Sequence<Row> run(final Query<Row> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
|
|
||||||
ListenableFuture<Sequence<Row>> future = queryExecutor.submit(
|
ListenableFuture<Sequence<Row>> future = queryExecutor.submit(
|
||||||
|
@ -112,7 +112,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
||||||
public Sequence<Row> call() throws Exception
|
public Sequence<Row> call() throws Exception
|
||||||
{
|
{
|
||||||
return new ExecutorExecutingSequence<Row>(
|
return new ExecutorExecutingSequence<Row>(
|
||||||
input.run(query, metadata),
|
input.run(query, context),
|
||||||
queryExecutor
|
queryExecutor
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -168,7 +168,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Row> run(Query<Row> input, Map<String, List> metadata)
|
public Sequence<Row> run(Query<Row> input, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
if (!(input instanceof GroupByQuery)) {
|
if (!(input instanceof GroupByQuery)) {
|
||||||
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class);
|
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), GroupByQuery.class);
|
||||||
|
|
|
@ -45,7 +45,6 @@ import io.druid.segment.QueryableIndex;
|
||||||
import io.druid.segment.Segment;
|
import io.druid.segment.Segment;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.CancellationException;
|
import java.util.concurrent.CancellationException;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
@ -75,7 +74,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
|
||||||
return new QueryRunner<SegmentAnalysis>()
|
return new QueryRunner<SegmentAnalysis>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence<SegmentAnalysis> run(Query<SegmentAnalysis> inQ, Map<String, List> metadata)
|
public Sequence<SegmentAnalysis> run(Query<SegmentAnalysis> inQ, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
SegmentMetadataQuery query = (SegmentMetadataQuery) inQ;
|
SegmentMetadataQuery query = (SegmentMetadataQuery) inQ;
|
||||||
|
|
||||||
|
@ -136,7 +135,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
|
||||||
@Override
|
@Override
|
||||||
public Sequence<SegmentAnalysis> run(
|
public Sequence<SegmentAnalysis> run(
|
||||||
final Query<SegmentAnalysis> query,
|
final Query<SegmentAnalysis> query,
|
||||||
final Map<String, List> metadata
|
final Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final int priority = query.getContextPriority(0);
|
final int priority = query.getContextPriority(0);
|
||||||
|
@ -146,7 +145,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
|
||||||
@Override
|
@Override
|
||||||
public Sequence<SegmentAnalysis> call() throws Exception
|
public Sequence<SegmentAnalysis> call() throws Exception
|
||||||
{
|
{
|
||||||
return input.run(query, metadata);
|
return input.run(query, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
|
@ -286,7 +286,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<SearchResultValue>> run(
|
public Sequence<Result<SearchResultValue>> run(
|
||||||
Query<Result<SearchResultValue>> input,
|
Query<Result<SearchResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof SearchQuery)) {
|
if (!(input instanceof SearchQuery)) {
|
||||||
|
@ -295,13 +295,13 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
||||||
|
|
||||||
final SearchQuery query = (SearchQuery) input;
|
final SearchQuery query = (SearchQuery) input;
|
||||||
if (query.getLimit() < config.getMaxSearchLimit()) {
|
if (query.getLimit() < config.getMaxSearchLimit()) {
|
||||||
return runner.run(query, metadata);
|
return runner.run(query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
final boolean isBySegment = query.getContextBySegment(false);
|
final boolean isBySegment = query.getContextBySegment(false);
|
||||||
|
|
||||||
return Sequences.map(
|
return Sequences.map(
|
||||||
runner.run(query.withLimit(config.getMaxSearchLimit()), metadata),
|
runner.run(query.withLimit(config.getMaxSearchLimit()), context),
|
||||||
new Function<Result<SearchResultValue>, Result<SearchResultValue>>()
|
new Function<Result<SearchResultValue>, Result<SearchResultValue>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -71,7 +71,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<SearchResultValue>> run(
|
public Sequence<Result<SearchResultValue>> run(
|
||||||
final Query<Result<SearchResultValue>> input,
|
final Query<Result<SearchResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof SearchQuery)) {
|
if (!(input instanceof SearchQuery)) {
|
||||||
|
|
|
@ -92,7 +92,7 @@ public class SelectQueryRunnerFactory
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<SelectResultValue>> run(
|
public Sequence<Result<SelectResultValue>> run(
|
||||||
Query<Result<SelectResultValue>> input,
|
Query<Result<SelectResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof SelectQuery)) {
|
if (!(input instanceof SelectQuery)) {
|
||||||
|
|
|
@ -51,7 +51,7 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> input, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> input, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final Query<T> query = input.withQuerySegmentSpec(specificSpec);
|
final Query<T> query = input.withQuerySegmentSpec(specificSpec);
|
||||||
|
|
||||||
|
@ -66,9 +66,9 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
|
||||||
{
|
{
|
||||||
Sequence<T> returningSeq;
|
Sequence<T> returningSeq;
|
||||||
try {
|
try {
|
||||||
returningSeq = base.run(query, metadata);
|
returningSeq = base.run(query, context);
|
||||||
} catch (NullStorageAdapterException e) {
|
} catch (NullStorageAdapterException e) {
|
||||||
metadata.get("missingSegments").add(((SpecificSegmentSpec) specificSpec).getDescriptor());
|
((List)context.get("missingSegments")).add(((SpecificSegmentSpec) specificSpec).getDescriptor());
|
||||||
returningSeq = Sequences.empty();
|
returningSeq = Sequences.empty();
|
||||||
}
|
}
|
||||||
return returningSeq;
|
return returningSeq;
|
||||||
|
|
|
@ -96,13 +96,13 @@ public class TimeBoundaryQueryQueryToolChest
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
protected Sequence<Result<TimeBoundaryResultValue>> doRun(
|
protected Sequence<Result<TimeBoundaryResultValue>> doRun(
|
||||||
QueryRunner<Result<TimeBoundaryResultValue>> baseRunner, Query<Result<TimeBoundaryResultValue>> input, Map<String, List> metadata
|
QueryRunner<Result<TimeBoundaryResultValue>> baseRunner, Query<Result<TimeBoundaryResultValue>> input, Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
TimeBoundaryQuery query = (TimeBoundaryQuery) input;
|
TimeBoundaryQuery query = (TimeBoundaryQuery) input;
|
||||||
return Sequences.simple(
|
return Sequences.simple(
|
||||||
query.mergeResults(
|
query.mergeResults(
|
||||||
Sequences.toList(baseRunner.run(query, metadata), Lists.<Result<TimeBoundaryResultValue>>newArrayList())
|
Sequences.toList(baseRunner.run(query, context), Lists.<Result<TimeBoundaryResultValue>>newArrayList())
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -86,7 +86,7 @@ public class TimeBoundaryQueryRunnerFactory
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<TimeBoundaryResultValue>> run(
|
public Sequence<Result<TimeBoundaryResultValue>> run(
|
||||||
Query<Result<TimeBoundaryResultValue>> input,
|
Query<Result<TimeBoundaryResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof TimeBoundaryQuery)) {
|
if (!(input instanceof TimeBoundaryQuery)) {
|
||||||
|
|
|
@ -93,7 +93,7 @@ public class TimeseriesQueryRunnerFactory
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<TimeseriesResultValue>> run(
|
public Sequence<Result<TimeseriesResultValue>> run(
|
||||||
Query<Result<TimeseriesResultValue>> input,
|
Query<Result<TimeseriesResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof TimeseriesQuery)) {
|
if (!(input instanceof TimeseriesQuery)) {
|
||||||
|
|
|
@ -412,7 +412,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<TopNResultValue>> run(
|
public Sequence<Result<TopNResultValue>> run(
|
||||||
Query<Result<TopNResultValue>> input,
|
Query<Result<TopNResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof TopNQuery)) {
|
if (!(input instanceof TopNQuery)) {
|
||||||
|
@ -421,13 +421,13 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
||||||
|
|
||||||
final TopNQuery query = (TopNQuery) input;
|
final TopNQuery query = (TopNQuery) input;
|
||||||
if (query.getThreshold() > minTopNThreshold) {
|
if (query.getThreshold() > minTopNThreshold) {
|
||||||
return runner.run(query, metadata);
|
return runner.run(query, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
final boolean isBySegment = query.getContextBySegment(false);
|
final boolean isBySegment = query.getContextBySegment(false);
|
||||||
|
|
||||||
return Sequences.map(
|
return Sequences.map(
|
||||||
runner.run(query.withThreshold(minTopNThreshold), metadata),
|
runner.run(query.withThreshold(minTopNThreshold), context),
|
||||||
new Function<Result<TopNResultValue>, Result<TopNResultValue>>()
|
new Function<Result<TopNResultValue>, Result<TopNResultValue>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -67,7 +67,7 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory<Result<TopNRes
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Result<TopNResultValue>> run(
|
public Sequence<Result<TopNResultValue>> run(
|
||||||
Query<Result<TopNResultValue>> input,
|
Query<Result<TopNResultValue>> input,
|
||||||
Map<String, List> metadata
|
Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (!(input instanceof TopNQuery)) {
|
if (!(input instanceof TopNQuery)) {
|
||||||
|
|
|
@ -36,7 +36,6 @@ import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
@ -102,14 +101,14 @@ public class ChainedExecutionQueryRunnerTest
|
||||||
runner3
|
runner3
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
final Sequence seq = chainedRunner.run(
|
final Sequence seq = chainedRunner.run(
|
||||||
Druids.newTimeseriesQueryBuilder()
|
Druids.newTimeseriesQueryBuilder()
|
||||||
.dataSource("test")
|
.dataSource("test")
|
||||||
.intervals("2014/2015")
|
.intervals("2014/2015")
|
||||||
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
|
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
);
|
);
|
||||||
|
|
||||||
Future resultFuture = Executors.newFixedThreadPool(1).submit(
|
Future resultFuture = Executors.newFixedThreadPool(1).submit(
|
||||||
|
@ -206,7 +205,7 @@ public class ChainedExecutionQueryRunnerTest
|
||||||
runner3
|
runner3
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
final Sequence seq = chainedRunner.run(
|
final Sequence seq = chainedRunner.run(
|
||||||
Druids.newTimeseriesQueryBuilder()
|
Druids.newTimeseriesQueryBuilder()
|
||||||
.dataSource("test")
|
.dataSource("test")
|
||||||
|
@ -214,7 +213,7 @@ public class ChainedExecutionQueryRunnerTest
|
||||||
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
|
.aggregators(Lists.<AggregatorFactory>newArrayList(new CountAggregatorFactory("count")))
|
||||||
.context(ImmutableMap.<String, Object>of("timeout", (100), "queryId", "test"))
|
.context(ImmutableMap.<String, Object>of("timeout", (100), "queryId", "test"))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
);
|
);
|
||||||
|
|
||||||
Future resultFuture = Executors.newFixedThreadPool(1).submit(
|
Future resultFuture = Executors.newFixedThreadPool(1).submit(
|
||||||
|
@ -268,7 +267,7 @@ public class ChainedExecutionQueryRunnerTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<Integer> run(Query<Integer> query, Map<String, List> metadata)
|
public Sequence<Integer> run(Query<Integer> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
hasStarted = true;
|
hasStarted = true;
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
|
|
|
@ -344,7 +344,7 @@ public class GroupByQueryRunnerTest
|
||||||
new QueryRunner<Row>()
|
new QueryRunner<Row>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence run(Query<Row> query, Map<String, List> metadata)
|
public Sequence run(Query<Row> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
// simulate two daily segments
|
// simulate two daily segments
|
||||||
final Query query1 = query.withQuerySegmentSpec(
|
final Query query1 = query.withQuerySegmentSpec(
|
||||||
|
@ -353,7 +353,7 @@ public class GroupByQueryRunnerTest
|
||||||
final Query query2 = query.withQuerySegmentSpec(
|
final Query query2 = query.withQuerySegmentSpec(
|
||||||
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
||||||
);
|
);
|
||||||
return Sequences.concat(runner.run(query1, metadata), runner.run(query2, metadata));
|
return Sequences.concat(runner.run(query1, context), runner.run(query2, context));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -370,9 +370,9 @@ public class GroupByQueryRunnerTest
|
||||||
createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L)
|
createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L)
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, metadata), "direct");
|
TestHelper.assertExpectedObjects(expectedResults, runner.run(fullQuery, context), "direct");
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged");
|
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
|
||||||
|
|
||||||
List<Row> allGranExpectedResults = Arrays.asList(
|
List<Row> allGranExpectedResults = Arrays.asList(
|
||||||
createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L),
|
createExpectedRow("2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L),
|
||||||
|
@ -386,8 +386,8 @@ public class GroupByQueryRunnerTest
|
||||||
createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L)
|
createExpectedRow("2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L)
|
||||||
);
|
);
|
||||||
|
|
||||||
TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, metadata), "direct");
|
TestHelper.assertExpectedObjects(allGranExpectedResults, runner.run(allGranQuery, context), "direct");
|
||||||
TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, metadata), "merged");
|
TestHelper.assertExpectedObjects(allGranExpectedResults, mergedRunner.run(allGranQuery, context), "merged");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -429,9 +429,9 @@ public class GroupByQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(
|
TestHelper.assertExpectedObjects(
|
||||||
Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, metadata), String.format("limit: %d", limit)
|
Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery, context), String.format("limit: %d", limit)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -537,7 +537,7 @@ public class GroupByQueryRunnerTest
|
||||||
new QueryRunner<Row>()
|
new QueryRunner<Row>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence run(Query<Row> query, Map<String, List> metadata)
|
public Sequence run(Query<Row> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
// simulate two daily segments
|
// simulate two daily segments
|
||||||
final Query query1 = query.withQuerySegmentSpec(
|
final Query query1 = query.withQuerySegmentSpec(
|
||||||
|
@ -546,12 +546,12 @@ public class GroupByQueryRunnerTest
|
||||||
final Query query2 = query.withQuerySegmentSpec(
|
final Query query2 = query.withQuerySegmentSpec(
|
||||||
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
||||||
);
|
);
|
||||||
return Sequences.concat(runner.run(query1, metadata), runner.run(query2, metadata));
|
return Sequences.concat(runner.run(query1, context), runner.run(query2, context));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged");
|
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -587,11 +587,11 @@ public class GroupByQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit");
|
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||||
|
|
||||||
TestHelper.assertExpectedObjects(
|
TestHelper.assertExpectedObjects(
|
||||||
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), metadata), "limited"
|
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -628,10 +628,10 @@ public class GroupByQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit");
|
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||||
TestHelper.assertExpectedObjects(
|
TestHelper.assertExpectedObjects(
|
||||||
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), metadata), "limited"
|
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -668,10 +668,10 @@ public class GroupByQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
QueryRunner<Row> mergeRunner = factory.getToolchest().mergeResults(runner);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit");
|
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||||
TestHelper.assertExpectedObjects(
|
TestHelper.assertExpectedObjects(
|
||||||
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), metadata), "limited"
|
Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build(), context), "limited"
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -711,7 +711,7 @@ public class GroupByQueryRunnerTest
|
||||||
new QueryRunner<Row>()
|
new QueryRunner<Row>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence run(Query<Row> query, Map<String, List> metadata)
|
public Sequence run(Query<Row> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
// simulate two daily segments
|
// simulate two daily segments
|
||||||
final Query query1 = query.withQuerySegmentSpec(
|
final Query query1 = query.withQuerySegmentSpec(
|
||||||
|
@ -720,13 +720,13 @@ public class GroupByQueryRunnerTest
|
||||||
final Query query2 = query.withQuerySegmentSpec(
|
final Query query2 = query.withQuerySegmentSpec(
|
||||||
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
||||||
);
|
);
|
||||||
return Sequences.concat(runner.run(query1, metadata), runner.run(query2, metadata));
|
return Sequences.concat(runner.run(query1, context), runner.run(query2, context));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, metadata), "merged");
|
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -765,9 +765,9 @@ public class GroupByQueryRunnerTest
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner);
|
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner);
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit");
|
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -799,8 +799,8 @@ public class GroupByQueryRunnerTest
|
||||||
createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L),
|
createExpectedRow("2011-04-01", "quality", "technology", "rows", 2L),
|
||||||
createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
|
createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, metadata), "normal");
|
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal");
|
||||||
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
||||||
configSupplier,
|
configSupplier,
|
||||||
new StupidPool<ByteBuffer>(
|
new StupidPool<ByteBuffer>(
|
||||||
|
@ -816,7 +816,7 @@ public class GroupByQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner);
|
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner);
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit");
|
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -849,8 +849,8 @@ public class GroupByQueryRunnerTest
|
||||||
createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
|
createExpectedRow("2011-04-01", "quality", "travel", "rows", 2L)
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, metadata), "normal");
|
TestHelper.assertExpectedObjects(expectedResults, runner.run(query, context), "normal");
|
||||||
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
final GroupByQueryEngine engine = new GroupByQueryEngine(
|
||||||
configSupplier,
|
configSupplier,
|
||||||
new StupidPool<ByteBuffer>(
|
new StupidPool<ByteBuffer>(
|
||||||
|
@ -866,7 +866,7 @@ public class GroupByQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner);
|
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner);
|
||||||
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, metadata), "no-limit");
|
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query, context), "no-limit");
|
||||||
}
|
}
|
||||||
|
|
||||||
// A subquery identical to the query should yield identical results
|
// A subquery identical to the query should yield identical results
|
||||||
|
@ -1046,8 +1046,8 @@ public class GroupByQueryRunnerTest
|
||||||
toolChest
|
toolChest
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Sequence<Row> queryResult = theRunner.run(query, metadata);
|
Sequence<Row> queryResult = theRunner.run(query, context);
|
||||||
return Sequences.toList(queryResult, Lists.<Row>newArrayList());
|
return Sequences.toList(queryResult, Lists.<Row>newArrayList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -102,7 +102,7 @@ public class SegmentAnalyzerTest
|
||||||
final SegmentMetadataQuery query = new SegmentMetadataQuery(
|
final SegmentMetadataQuery query = new SegmentMetadataQuery(
|
||||||
new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null
|
new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
return Sequences.toList(query.run(runner, metadata), Lists.<SegmentAnalysis>newArrayList());
|
return Sequences.toList(query.run(runner, context), Lists.<SegmentAnalysis>newArrayList());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -72,9 +72,9 @@ public class SegmentMetadataQueryTest
|
||||||
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
|
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
|
||||||
.merge(true)
|
.merge(true)
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<SegmentAnalysis> results = Sequences.toList(
|
Iterable<SegmentAnalysis> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<SegmentAnalysis>newArrayList()
|
Lists.<SegmentAnalysis>newArrayList()
|
||||||
);
|
);
|
||||||
SegmentAnalysis val = results.iterator().next();
|
SegmentAnalysis val = results.iterator().next();
|
||||||
|
|
|
@ -369,9 +369,9 @@ public class SearchQueryRunnerTest
|
||||||
|
|
||||||
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
|
private void checkSearchQuery(SearchQuery searchQuery, Map<String, Set<String>> expectedResults)
|
||||||
{
|
{
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
Iterable<Result<SearchResultValue>> results = Sequences.toList(
|
Iterable<Result<SearchResultValue>> results = Sequences.toList(
|
||||||
runner.run(searchQuery, metadata),
|
runner.run(searchQuery, context),
|
||||||
Lists.<Result<SearchResultValue>>newArrayList()
|
Lists.<Result<SearchResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -88,9 +88,9 @@ public class SelectQueryRunnerTest
|
||||||
new PagingSpec(null, 3),
|
new PagingSpec(null, 3),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<SelectResultValue>>newArrayList()
|
Lists.<Result<SelectResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -157,9 +157,9 @@ public class SelectQueryRunnerTest
|
||||||
new PagingSpec(null, 3),
|
new PagingSpec(null, 3),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<SelectResultValue>>newArrayList()
|
Lists.<Result<SelectResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -217,9 +217,9 @@ public class SelectQueryRunnerTest
|
||||||
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
|
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<SelectResultValue>>newArrayList()
|
Lists.<Result<SelectResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -277,9 +277,9 @@ public class SelectQueryRunnerTest
|
||||||
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
|
new PagingSpec(Maps.newLinkedHashMap(ImmutableMap.of(QueryRunnerTestHelper.segmentId, 3)), 3),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
Iterable<Result<SelectResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<SelectResultValue>>newArrayList()
|
Lists.<Result<SelectResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.junit.runners.Parameterized;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -65,9 +64,9 @@ public class TimeBoundaryQueryRunnerTest
|
||||||
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder()
|
||||||
.dataSource("testing")
|
.dataSource("testing")
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeBoundaryResultValue>> results = Sequences.toList(
|
||||||
runner.run(timeBoundaryQuery, metadata),
|
runner.run(timeBoundaryQuery, context),
|
||||||
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
Lists.<Result<TimeBoundaryResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TimeBoundaryResultValue val = results.iterator().next().getValue();
|
TimeBoundaryResultValue val = results.iterator().next().getValue();
|
||||||
|
|
|
@ -98,9 +98,9 @@ public class TimeSeriesUnionQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -111,9 +111,9 @@ public class TimeseriesQueryRunnerBonusTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
return Sequences.toList(
|
return Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -101,10 +101,10 @@ public class TimeseriesQueryRunnerTest
|
||||||
|
|
||||||
DateTime expectedEarliest = new DateTime("2011-01-12");
|
DateTime expectedEarliest = new DateTime("2011-01-12");
|
||||||
DateTime expectedLast = new DateTime("2011-04-15");
|
DateTime expectedLast = new DateTime("2011-04-15");
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
|
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -168,9 +168,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
DateTime expectedEarliest = new DateTime("2011-01-12");
|
DateTime expectedEarliest = new DateTime("2011-01-12");
|
||||||
DateTime expectedLast = new DateTime("2011-04-15");
|
DateTime expectedLast = new DateTime("2011-04-15");
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
Result<TimeseriesResultValue> result = results.iterator().next();
|
Result<TimeseriesResultValue> result = results.iterator().next();
|
||||||
|
@ -215,9 +215,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
|
|
||||||
DateTime expectedEarliest = new DateTime("2011-01-12");
|
DateTime expectedEarliest = new DateTime("2011-01-12");
|
||||||
DateTime expectedLast = new DateTime("2011-04-15");
|
DateTime expectedLast = new DateTime("2011-04-15");
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -281,9 +281,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -328,9 +328,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -370,9 +370,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
|
||||||
runner.run(query1, metadata),
|
runner.run(query1, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults1, results1);
|
TestHelper.assertExpectedResults(expectedResults1, results1);
|
||||||
|
@ -409,7 +409,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
|
||||||
runner.run(query2, metadata),
|
runner.run(query2, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults2, results2);
|
TestHelper.assertExpectedResults(expectedResults2, results2);
|
||||||
|
@ -460,9 +460,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
|
||||||
runner.run(query1, metadata),
|
runner.run(query1, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults1, results1);
|
TestHelper.assertExpectedResults(expectedResults1, results1);
|
||||||
|
@ -502,9 +502,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results1 = Sequences.toList(
|
||||||
runner.run(query1, metadata),
|
runner.run(query1, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults1, results1);
|
TestHelper.assertExpectedResults(expectedResults1, results1);
|
||||||
|
@ -542,7 +542,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
);
|
);
|
||||||
|
|
||||||
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results2 = Sequences.toList(
|
||||||
runner.run(query2, metadata),
|
runner.run(query2, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults2, results2);
|
TestHelper.assertExpectedResults(expectedResults2, results2);
|
||||||
|
@ -575,9 +575,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList();
|
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -625,9 +625,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -675,9 +675,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -725,9 +725,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -775,9 +775,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -825,9 +825,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -883,9 +883,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -941,9 +941,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -999,9 +999,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -1055,9 +1055,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -1117,9 +1117,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -1161,9 +1161,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -1205,9 +1205,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -1263,9 +1263,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, results);
|
TestHelper.assertExpectedResults(expectedResults, results);
|
||||||
|
@ -1303,9 +1303,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
||||||
|
@ -1344,9 +1344,9 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
||||||
|
@ -1363,7 +1363,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
||||||
runner.run(
|
runner.run(
|
||||||
Druids.newTimeseriesQueryBuilder()
|
Druids.newTimeseriesQueryBuilder()
|
||||||
|
@ -1373,12 +1373,12 @@ public class TimeseriesQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
),
|
),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
||||||
|
@ -1395,7 +1395,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
||||||
runner.run(
|
runner.run(
|
||||||
Druids.newTimeseriesQueryBuilder()
|
Druids.newTimeseriesQueryBuilder()
|
||||||
|
@ -1406,12 +1406,12 @@ public class TimeseriesQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
),
|
),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
||||||
|
@ -1457,7 +1457,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
||||||
runner.run(
|
runner.run(
|
||||||
Druids.newTimeseriesQueryBuilder()
|
Druids.newTimeseriesQueryBuilder()
|
||||||
|
@ -1468,12 +1468,12 @@ public class TimeseriesQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
),
|
),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
||||||
|
@ -1521,7 +1521,7 @@ public class TimeseriesQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> expectedResults = Sequences.toList(
|
||||||
runner.run(
|
runner.run(
|
||||||
Druids.newTimeseriesQueryBuilder()
|
Druids.newTimeseriesQueryBuilder()
|
||||||
|
@ -1532,12 +1532,12 @@ public class TimeseriesQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
),
|
),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
Iterable<Result<TimeseriesResultValue>> actualResults = Sequences.toList(
|
||||||
runner.run(query, metadata),
|
runner.run(query, context),
|
||||||
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
);
|
);
|
||||||
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
TestHelper.assertExpectedResults(expectedResults, actualResults);
|
||||||
|
|
|
@ -167,8 +167,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -231,8 +231,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -296,8 +296,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -346,8 +346,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -395,8 +395,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -444,8 +444,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -486,8 +486,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -521,8 +521,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -570,8 +570,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -623,8 +623,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -665,8 +665,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -683,7 +683,7 @@ public class TopNQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
Lists.<Result<TopNResultValue>>newArrayList(
|
Lists.<Result<TopNResultValue>>newArrayList(
|
||||||
new Result<TopNResultValue>(
|
new Result<TopNResultValue>(
|
||||||
|
@ -691,7 +691,7 @@ public class TopNQueryRunnerTest
|
||||||
new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
|
new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
runner.run(query, metadata)
|
runner.run(query, context)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -722,7 +722,7 @@ public class TopNQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
Lists.<Result<TopNResultValue>>newArrayList(
|
Lists.<Result<TopNResultValue>>newArrayList(
|
||||||
new Result<TopNResultValue>(
|
new Result<TopNResultValue>(
|
||||||
|
@ -730,7 +730,7 @@ public class TopNQueryRunnerTest
|
||||||
new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
|
new TopNResultValue(Lists.<Map<String, Object>>newArrayList())
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
runner.run(query, metadata)
|
runner.run(query, context)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -748,7 +748,7 @@ public class TopNQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
Sequences.toList(
|
Sequences.toList(
|
||||||
runner.run(
|
runner.run(
|
||||||
|
@ -763,9 +763,9 @@ public class TopNQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
), Lists.<Result<TopNResultValue>>newArrayList()
|
), Lists.<Result<TopNResultValue>>newArrayList()
|
||||||
), runner.run(query, metadata)
|
), runner.run(query, context)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -783,7 +783,7 @@ public class TopNQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build();
|
.build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
Sequences.toList(
|
Sequences.toList(
|
||||||
runner.run(
|
runner.run(
|
||||||
|
@ -798,10 +798,10 @@ public class TopNQueryRunnerTest
|
||||||
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
.aggregators(QueryRunnerTestHelper.commonAggregators)
|
||||||
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
), Lists.<Result<TopNResultValue>>newArrayList()
|
), Lists.<Result<TopNResultValue>>newArrayList()
|
||||||
)
|
)
|
||||||
, runner.run(query, metadata)
|
, runner.run(query, context)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -843,8 +843,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -892,8 +892,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -948,8 +948,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -996,8 +996,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1037,8 +1037,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1078,8 +1078,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1119,8 +1119,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1160,8 +1160,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1212,8 +1212,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1264,8 +1264,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1316,8 +1316,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1361,8 +1361,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1407,8 +1407,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1452,8 +1452,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1501,8 +1501,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1586,8 +1586,8 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1669,7 +1669,7 @@ public class TopNQueryRunnerTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -174,8 +174,8 @@ public class TopNUnionQueryTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -449,8 +449,8 @@ public class SpatialFilterBonusTest
|
||||||
factory.createRunner(segment),
|
factory.createRunner(segment),
|
||||||
factory.getToolchest()
|
factory.getToolchest()
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -536,8 +536,8 @@ public class SpatialFilterBonusTest
|
||||||
factory.createRunner(segment),
|
factory.createRunner(segment),
|
||||||
factory.getToolchest()
|
factory.getToolchest()
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
|
|
@ -479,8 +479,8 @@ public class SpatialFilterTest
|
||||||
factory.createRunner(segment),
|
factory.createRunner(segment),
|
||||||
factory.getToolchest()
|
factory.getToolchest()
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -566,8 +566,8 @@ public class SpatialFilterTest
|
||||||
factory.createRunner(segment),
|
factory.createRunner(segment),
|
||||||
factory.getToolchest()
|
factory.getToolchest()
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
TestHelper.assertExpectedResults(expectedResults, runner.run(query, metadata));
|
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
|
|
@ -115,7 +115,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
||||||
final CacheStrategy<T, Object, Query<T>> strategy = toolChest.getCacheStrategy(query);
|
final CacheStrategy<T, Object, Query<T>> strategy = toolChest.getCacheStrategy(query);
|
||||||
|
@ -327,11 +327,11 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
||||||
List<Interval> intervals = segmentSpec.getIntervals();
|
List<Interval> intervals = segmentSpec.getIntervals();
|
||||||
|
|
||||||
if (!server.isAssignable() || !populateCache || isBySegment) {
|
if (!server.isAssignable() || !populateCache || isBySegment) {
|
||||||
resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), metadata);
|
resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), context);
|
||||||
} else {
|
} else {
|
||||||
resultSeqToAdd = toolChest.mergeSequences(
|
resultSeqToAdd = toolChest.mergeSequences(
|
||||||
Sequences.map(
|
Sequences.map(
|
||||||
clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec), metadata),
|
clientQueryable.run(rewrittenQuery.withQuerySegmentSpec(segmentSpec), context),
|
||||||
new Function<Object, Sequence<T>>()
|
new Function<Object, Sequence<T>>()
|
||||||
{
|
{
|
||||||
private final Function<T, Object> cacheFn = strategy.prepareForCache();
|
private final Function<T, Object> cacheFn = strategy.prepareForCache();
|
||||||
|
|
|
@ -73,7 +73,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
final CacheStrategy strategy = toolChest.getCacheStrategy(query);
|
final CacheStrategy strategy = toolChest.getCacheStrategy(query);
|
||||||
|
|
||||||
|
@ -141,7 +141,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
|
||||||
|
|
||||||
return Sequences.withEffect(
|
return Sequences.withEffect(
|
||||||
Sequences.map(
|
Sequences.map(
|
||||||
base.run(query, metadata),
|
base.run(query, context),
|
||||||
new Function<T, T>()
|
new Function<T, T>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -163,7 +163,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
|
||||||
MoreExecutors.sameThreadExecutor()
|
MoreExecutors.sameThreadExecutor()
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
return base.run(query, metadata);
|
return base.run(query, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -114,7 +114,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(final Query<T> query, final Map<String, List> metadata)
|
public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
|
||||||
{
|
{
|
||||||
QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
||||||
boolean isBySegment = query.getContextBySegment(false);
|
boolean isBySegment = query.getContextBySegment(false);
|
||||||
|
@ -170,7 +170,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
}
|
}
|
||||||
metadata.get("missingSegments").addAll(missingSegments);
|
((List) context.get("missingSegments")).addAll(missingSegments);
|
||||||
}
|
}
|
||||||
|
|
||||||
return super.handleResponse(response);
|
return super.handleResponse(response);
|
||||||
|
|
|
@ -142,9 +142,9 @@ public class QueryResource
|
||||||
log.debug("Got query [%s]", query);
|
log.debug("Got query [%s]", query);
|
||||||
}
|
}
|
||||||
|
|
||||||
HashMap<String, List> metadata = new HashMap<String, List>();
|
HashMap<String, Object> context = new HashMap<String, Object>();
|
||||||
metadata.put("missingSegments", new LinkedList());
|
context.put("missingSegments", new LinkedList());
|
||||||
Sequence results = query.run(texasRanger, metadata);
|
Sequence results = query.run(texasRanger, context);
|
||||||
|
|
||||||
if (results == null) {
|
if (results == null) {
|
||||||
results = Sequences.empty();
|
results = Sequences.empty();
|
||||||
|
@ -166,8 +166,8 @@ public class QueryResource
|
||||||
) {
|
) {
|
||||||
|
|
||||||
String missingSegments = "";
|
String missingSegments = "";
|
||||||
if (!metadata.get("missingSegments").isEmpty()) {
|
if (!((List)context.get("missingSegments")).isEmpty()) {
|
||||||
missingSegments = jsonMapper.writeValueAsString(metadata.get("missingSegments"));
|
missingSegments = jsonMapper.writeValueAsString(context.get("missingSegments"));
|
||||||
}
|
}
|
||||||
|
|
||||||
long requestTime = System.currentTimeMillis() - start;
|
long requestTime = System.currentTimeMillis() - start;
|
||||||
|
|
|
@ -87,7 +87,7 @@ public class BridgeQuerySegmentWalker implements QuerySegmentWalker
|
||||||
return new QueryRunner<T>()
|
return new QueryRunner<T>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
Server instance = brokerSelector.pick();
|
Server instance = brokerSelector.pick();
|
||||||
|
|
|
@ -263,7 +263,7 @@ public class CachingClusteredClientTest
|
||||||
new DateTime("2011-01-09T01"), 181, 52
|
new DateTime("2011-01-09T01"), 181, 52
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
makeRenamedTimeResults(
|
makeRenamedTimeResults(
|
||||||
new DateTime("2011-01-01"), 50, 5000,
|
new DateTime("2011-01-01"), 50, 5000,
|
||||||
|
@ -285,7 +285,7 @@ public class CachingClusteredClientTest
|
||||||
.aggregators(RENAMED_AGGS)
|
.aggregators(RENAMED_AGGS)
|
||||||
.postAggregators(RENAMED_POST_AGGS)
|
.postAggregators(RENAMED_POST_AGGS)
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -316,7 +316,7 @@ public class CachingClusteredClientTest
|
||||||
new DateTime("2011-11-07", TIMEZONE), 85, 102
|
new DateTime("2011-11-07", TIMEZONE), 85, 102
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
makeRenamedTimeResults(
|
makeRenamedTimeResults(
|
||||||
new DateTime("2011-11-04", TIMEZONE), 50, 5000,
|
new DateTime("2011-11-04", TIMEZONE), 50, 5000,
|
||||||
|
@ -329,7 +329,7 @@ public class CachingClusteredClientTest
|
||||||
.aggregators(RENAMED_AGGS)
|
.aggregators(RENAMED_AGGS)
|
||||||
.postAggregators(RENAMED_POST_AGGS)
|
.postAggregators(RENAMED_POST_AGGS)
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -445,7 +445,7 @@ public class CachingClusteredClientTest
|
||||||
new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
|
new DateTime("2011-01-09T01"), "c2", 50, 4985, "b", 50, 4984, "c", 50, 4983
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
makeRenamedTopNResults(
|
makeRenamedTopNResults(
|
||||||
new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998,
|
new DateTime("2011-01-01"), "a", 50, 5000, "b", 50, 4999, "c", 50, 4998,
|
||||||
|
@ -467,7 +467,7 @@ public class CachingClusteredClientTest
|
||||||
.aggregators(RENAMED_AGGS)
|
.aggregators(RENAMED_AGGS)
|
||||||
.postAggregators(RENAMED_POST_AGGS)
|
.postAggregators(RENAMED_POST_AGGS)
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -501,7 +501,7 @@ public class CachingClusteredClientTest
|
||||||
new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986
|
new DateTime("2011-11-07", TIMEZONE), "a", 50, 4988, "b", 50, 4987, "c", 50, 4986
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
makeRenamedTopNResults(
|
makeRenamedTopNResults(
|
||||||
|
|
||||||
|
@ -516,7 +516,7 @@ public class CachingClusteredClientTest
|
||||||
.aggregators(RENAMED_AGGS)
|
.aggregators(RENAMED_AGGS)
|
||||||
.postAggregators(RENAMED_POST_AGGS)
|
.postAggregators(RENAMED_POST_AGGS)
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -566,7 +566,7 @@ public class CachingClusteredClientTest
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
makeRenamedTopNResults(
|
makeRenamedTopNResults(
|
||||||
new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992,
|
new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992,
|
||||||
|
@ -586,7 +586,7 @@ public class CachingClusteredClientTest
|
||||||
.aggregators(RENAMED_AGGS)
|
.aggregators(RENAMED_AGGS)
|
||||||
.postAggregators(RENAMED_POST_AGGS)
|
.postAggregators(RENAMED_POST_AGGS)
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -635,7 +635,7 @@ public class CachingClusteredClientTest
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
makeTopNResults(
|
makeTopNResults(
|
||||||
new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992,
|
new DateTime("2011-01-05"), "a", 50, 4994, "b", 50, 4993, "c", 50, 4992,
|
||||||
|
@ -655,7 +655,7 @@ public class CachingClusteredClientTest
|
||||||
.aggregators(AGGS)
|
.aggregators(AGGS)
|
||||||
.postAggregators(POST_AGGS)
|
.postAggregators(POST_AGGS)
|
||||||
.build(),
|
.build(),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -763,7 +763,7 @@ public class CachingClusteredClientTest
|
||||||
.once();
|
.once();
|
||||||
|
|
||||||
final Capture<? extends Query> capture = new Capture();
|
final Capture<? extends Query> capture = new Capture();
|
||||||
final Capture<? extends Map> metadata = new Capture();
|
final Capture<? extends Map> context = new Capture();
|
||||||
queryCaptures.add(capture);
|
queryCaptures.add(capture);
|
||||||
QueryRunner queryable = expectations.getQueryRunner();
|
QueryRunner queryable = expectations.getQueryRunner();
|
||||||
|
|
||||||
|
@ -776,7 +776,7 @@ public class CachingClusteredClientTest
|
||||||
intervals.add(expectation.getInterval());
|
intervals.add(expectation.getInterval());
|
||||||
results.add(expectation.getResults());
|
results.add(expectation.getResults());
|
||||||
}
|
}
|
||||||
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata)))
|
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context)))
|
||||||
.andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results))
|
.andReturn(toQueryableTimeseriesResults(expectBySegment, segmentIds, intervals, results))
|
||||||
.once();
|
.once();
|
||||||
|
|
||||||
|
@ -789,7 +789,7 @@ public class CachingClusteredClientTest
|
||||||
intervals.add(expectation.getInterval());
|
intervals.add(expectation.getInterval());
|
||||||
results.add(expectation.getResults());
|
results.add(expectation.getResults());
|
||||||
}
|
}
|
||||||
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata)))
|
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context)))
|
||||||
.andReturn(toQueryableTopNResults(segmentIds, intervals, results))
|
.andReturn(toQueryableTopNResults(segmentIds, intervals, results))
|
||||||
.once();
|
.once();
|
||||||
} else if (query instanceof SearchQuery) {
|
} else if (query instanceof SearchQuery) {
|
||||||
|
@ -801,7 +801,7 @@ public class CachingClusteredClientTest
|
||||||
intervals.add(expectation.getInterval());
|
intervals.add(expectation.getInterval());
|
||||||
results.add(expectation.getResults());
|
results.add(expectation.getResults());
|
||||||
}
|
}
|
||||||
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata)))
|
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context)))
|
||||||
.andReturn(toQueryableSearchResults(segmentIds, intervals, results))
|
.andReturn(toQueryableSearchResults(segmentIds, intervals, results))
|
||||||
.once();
|
.once();
|
||||||
} else if (query instanceof TimeBoundaryQuery) {
|
} else if (query instanceof TimeBoundaryQuery) {
|
||||||
|
@ -813,7 +813,7 @@ public class CachingClusteredClientTest
|
||||||
intervals.add(expectation.getInterval());
|
intervals.add(expectation.getInterval());
|
||||||
results.add(expectation.getResults());
|
results.add(expectation.getResults());
|
||||||
}
|
}
|
||||||
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(metadata)))
|
EasyMock.expect(queryable.run(EasyMock.capture(capture), EasyMock.capture(context)))
|
||||||
.andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results))
|
.andReturn(toQueryableTimeBoundaryResults(segmentIds, intervals, results))
|
||||||
.once();
|
.once();
|
||||||
} else {
|
} else {
|
||||||
|
@ -837,7 +837,7 @@ public class CachingClusteredClientTest
|
||||||
@Override
|
@Override
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
for (int i = 0; i < numTimesToQuery; ++i) {
|
for (int i = 0; i < numTimesToQuery; ++i) {
|
||||||
TestHelper.assertExpectedResults(
|
TestHelper.assertExpectedResults(
|
||||||
new MergeIterable<>(
|
new MergeIterable<>(
|
||||||
|
@ -872,7 +872,7 @@ public class CachingClusteredClientTest
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
metadata
|
context
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -124,7 +124,7 @@ public class CachingQueryRunnerTest
|
||||||
new QueryRunner()
|
new QueryRunner()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence run(Query query, Map metadata)
|
public Sequence run(Query query, Map context)
|
||||||
{
|
{
|
||||||
return resultSeq;
|
return resultSeq;
|
||||||
}
|
}
|
||||||
|
@ -141,8 +141,8 @@ public class CachingQueryRunnerTest
|
||||||
cacheStrategy.computeCacheKey(query)
|
cacheStrategy.computeCacheKey(query)
|
||||||
);
|
);
|
||||||
|
|
||||||
HashMap<String,Object> metadata = new HashMap<String, Object>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
Sequence res = runner.run(query, metadata);
|
Sequence res = runner.run(query, context);
|
||||||
// base sequence is not closed yet
|
// base sequence is not closed yet
|
||||||
Assert.assertFalse("sequence must not be closed", closable.isClosed());
|
Assert.assertFalse("sequence must not be closed", closable.isClosed());
|
||||||
Assert.assertNull("cache must be empty", cache.get(cacheKey));
|
Assert.assertNull("cache must be empty", cache.get(cacheKey));
|
||||||
|
@ -214,7 +214,7 @@ public class CachingQueryRunnerTest
|
||||||
new QueryRunner()
|
new QueryRunner()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Sequence run(Query query, Map metadata)
|
public Sequence run(Query query, Map context)
|
||||||
{
|
{
|
||||||
return Sequences.empty();
|
return Sequences.empty();
|
||||||
}
|
}
|
||||||
|
@ -222,8 +222,8 @@ public class CachingQueryRunnerTest
|
||||||
new CacheConfig()
|
new CacheConfig()
|
||||||
|
|
||||||
);
|
);
|
||||||
HashMap<String,Object> metadata = new HashMap<String, Object>();
|
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||||
List<Object> results = Sequences.toList(runner.run(query, metadata), new ArrayList());
|
List<Object> results = Sequences.toList(runner.run(query, context), new ArrayList());
|
||||||
Assert.assertEquals(expectedResults, results);
|
Assert.assertEquals(expectedResults, results);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -117,20 +117,20 @@ public class DirectDruidClientTest
|
||||||
serverSelector.addServer(queryableDruidServer2);
|
serverSelector.addServer(queryableDruidServer2);
|
||||||
|
|
||||||
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
|
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
Sequence s1 = client1.run(query, metadata);
|
Sequence s1 = client1.run(query, context);
|
||||||
Assert.assertEquals(1, client1.getNumOpenConnections());
|
Assert.assertEquals(1, client1.getNumOpenConnections());
|
||||||
|
|
||||||
// simulate read timeout
|
// simulate read timeout
|
||||||
Sequence s2 = client1.run(query, metadata);
|
Sequence s2 = client1.run(query, context);
|
||||||
Assert.assertEquals(2, client1.getNumOpenConnections());
|
Assert.assertEquals(2, client1.getNumOpenConnections());
|
||||||
futureException.setException(new ReadTimeoutException());
|
futureException.setException(new ReadTimeoutException());
|
||||||
Assert.assertEquals(1, client1.getNumOpenConnections());
|
Assert.assertEquals(1, client1.getNumOpenConnections());
|
||||||
|
|
||||||
// subsequent connections should work
|
// subsequent connections should work
|
||||||
Sequence s3 = client1.run(query, metadata);
|
Sequence s3 = client1.run(query, context);
|
||||||
Sequence s4 = client1.run(query, metadata);
|
Sequence s4 = client1.run(query, context);
|
||||||
Sequence s5 = client1.run(query, metadata);
|
Sequence s5 = client1.run(query, context);
|
||||||
|
|
||||||
Assert.assertTrue(client1.getNumOpenConnections() == 4);
|
Assert.assertTrue(client1.getNumOpenConnections() == 4);
|
||||||
|
|
||||||
|
@ -141,8 +141,8 @@ public class DirectDruidClientTest
|
||||||
Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp());
|
Assert.assertEquals(new DateTime("2014-01-01T01:02:03Z"), results.get(0).getTimestamp());
|
||||||
Assert.assertEquals(3, client1.getNumOpenConnections());
|
Assert.assertEquals(3, client1.getNumOpenConnections());
|
||||||
|
|
||||||
client2.run(query, metadata);
|
client2.run(query, context);
|
||||||
client2.run(query, metadata);
|
client2.run(query, context);
|
||||||
|
|
||||||
Assert.assertTrue(client2.getNumOpenConnections() == 2);
|
Assert.assertTrue(client2.getNumOpenConnections() == 2);
|
||||||
|
|
||||||
|
@ -200,9 +200,9 @@ public class DirectDruidClientTest
|
||||||
serverSelector.addServer(queryableDruidServer1);
|
serverSelector.addServer(queryableDruidServer1);
|
||||||
|
|
||||||
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
|
TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build();
|
||||||
HashMap<String,List> metadata = new HashMap<String, List>();
|
HashMap<String,List> context = new HashMap<String, List>();
|
||||||
cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled")));
|
cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled")));
|
||||||
Sequence results = client1.run(query, metadata);
|
Sequence results = client1.run(query, context);
|
||||||
Assert.assertEquals(0, client1.getNumOpenConnections());
|
Assert.assertEquals(0, client1.getNumOpenConnections());
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -429,8 +429,8 @@ public class ServerManagerTest
|
||||||
@Override
|
@Override
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
Map<String,List> metadata = new HashMap<String, List>();
|
Map<String,Object> context = new HashMap<String, Object>();
|
||||||
Sequence<Result<SearchResultValue>> seq = runner.run(query, metadata);
|
Sequence<Result<SearchResultValue>> seq = runner.run(query, context);
|
||||||
Sequences.toList(seq, Lists.<Result<SearchResultValue>>newArrayList());
|
Sequences.toList(seq, Lists.<Result<SearchResultValue>>newArrayList());
|
||||||
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
|
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
|
||||||
|
|
||||||
|
@ -679,9 +679,9 @@ public class ServerManagerTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Sequence<T> run(Query<T> query, Map<String, List> metadata)
|
public Sequence<T> run(Query<T> query, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
return new BlockingSequence<T>(runner.run(query, metadata), waitLatch, waitYieldLatch, notifyLatch);
|
return new BlockingSequence<T>(runner.run(query, context), waitLatch, waitYieldLatch, notifyLatch);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue