mirror of https://github.com/apache/druid.git
commit
7f15eb51f2
|
@ -25,6 +25,8 @@ import com.google.common.collect.Lists;
|
|||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.FinalizeMetricManipulationFn;
|
||||
import io.druid.query.aggregation.IdentityMetricManipulationFn;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -50,21 +52,19 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
|
|||
{
|
||||
final boolean isBySegment = query.getContextBySegment(false);
|
||||
final boolean shouldFinalize = query.getContextFinalize(true);
|
||||
|
||||
final Query<T> queryToRun;
|
||||
final Function<T, T> finalizerFn;
|
||||
|
||||
if (shouldFinalize) {
|
||||
Function<T, T> finalizerFn;
|
||||
queryToRun = query.withOverriddenContext(ImmutableMap.<String, Object>of("finalize", false));
|
||||
|
||||
if (isBySegment) {
|
||||
finalizerFn = new Function<T, T>()
|
||||
{
|
||||
final Function<T, T> baseFinalizer = toolChest.makeMetricManipulatorFn(
|
||||
final Function<T, T> baseFinalizer = toolChest.makePostComputeManipulatorFn(
|
||||
query,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.finalizeComputation(factory.deserialize(object));
|
||||
}
|
||||
}
|
||||
new FinalizeMetricManipulationFn()
|
||||
);
|
||||
|
||||
@Override
|
||||
|
@ -85,24 +85,21 @@ public class FinalizeResultsQueryRunner<T> implements QueryRunner<T>
|
|||
}
|
||||
};
|
||||
} else {
|
||||
finalizerFn = toolChest.makeMetricManipulatorFn(
|
||||
query,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.finalizeComputation(object);
|
||||
}
|
||||
}
|
||||
);
|
||||
finalizerFn = toolChest.makePostComputeManipulatorFn(query, new FinalizeMetricManipulationFn());
|
||||
}
|
||||
|
||||
return Sequences.map(
|
||||
baseRunner.run(query.withOverriddenContext(ImmutableMap.<String, Object>of("finalize", false))),
|
||||
finalizerFn
|
||||
} else {
|
||||
// finalize is false here.
|
||||
queryToRun = query;
|
||||
finalizerFn = toolChest.makePostComputeManipulatorFn(
|
||||
query,
|
||||
new IdentityMetricManipulationFn()
|
||||
);
|
||||
}
|
||||
return baseRunner.run(query);
|
||||
|
||||
return Sequences.map(
|
||||
baseRunner.run(queryToRun),
|
||||
finalizerFn
|
||||
);
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,8 +44,16 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
|
|||
* @return
|
||||
*/
|
||||
public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
|
||||
|
||||
public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
|
||||
public abstract Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
|
||||
|
||||
public abstract Function<ResultType, ResultType> makePreComputeManipulatorFn(QueryType query, MetricManipulationFn fn);
|
||||
|
||||
public Function<ResultType, ResultType> makePostComputeManipulatorFn(QueryType query, MetricManipulationFn fn)
|
||||
{
|
||||
return makePreComputeManipulatorFn(query, fn);
|
||||
}
|
||||
|
||||
public abstract TypeReference<ResultType> getResultTypeReference();
|
||||
|
||||
public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query) {
|
||||
|
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FinalizeMetricManipulationFn implements MetricManipulationFn
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.finalizeComputation(object);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IdentityMetricManipulationFn implements MetricManipulationFn
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return object;
|
||||
}
|
||||
}
|
|
@ -173,7 +173,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<Row, Row> makeMetricManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn)
|
||||
public Function<Row, Row> makePreComputeManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn)
|
||||
{
|
||||
return new Function<Row, Row>()
|
||||
{
|
||||
|
|
|
@ -155,7 +155,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentAnalysis, SegmentAnalysis> makeMetricManipulatorFn(
|
||||
public Function<SegmentAnalysis, SegmentAnalysis> makePreComputeManipulatorFn(
|
||||
SegmentMetadataQuery query, MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
|
|
|
@ -129,7 +129,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<SearchResultValue>, Result<SearchResultValue>> makeMetricManipulatorFn(
|
||||
public Function<Result<SearchResultValue>, Result<SearchResultValue>> makePreComputeManipulatorFn(
|
||||
SearchQuery query, MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
|
|
|
@ -131,7 +131,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<SelectResultValue>, Result<SelectResultValue>> makeMetricManipulatorFn(
|
||||
public Function<Result<SelectResultValue>, Result<SelectResultValue>> makePreComputeManipulatorFn(
|
||||
final SelectQuery query, final MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
|
|
|
@ -123,7 +123,7 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TimeBoundaryResultValue>, Result<TimeBoundaryResultValue>> makeMetricManipulatorFn(
|
||||
public Function<Result<TimeBoundaryResultValue>, Result<TimeBoundaryResultValue>> makePreComputeManipulatorFn(
|
||||
TimeBoundaryQuery query, MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
|
|
|
@ -130,32 +130,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
|
|||
.setUser9(Minutes.minutes(numMinutes).toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> makeMetricManipulatorFn(
|
||||
final TimeseriesQuery query, final MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
return new Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>>()
|
||||
{
|
||||
@Override
|
||||
public Result<TimeseriesResultValue> apply(Result<TimeseriesResultValue> result)
|
||||
{
|
||||
final Map<String, Object> values = Maps.newHashMap();
|
||||
final TimeseriesResultValue holder = result.getValue();
|
||||
for (AggregatorFactory agg : query.getAggregatorSpecs()) {
|
||||
values.put(agg.getName(), fn.manipulate(agg, holder.getMetric(agg.getName())));
|
||||
}
|
||||
for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
|
||||
values.put(postAgg.getName(), postAgg.compute(values));
|
||||
}
|
||||
return new Result<TimeseriesResultValue>(
|
||||
result.getTimestamp(),
|
||||
new TimeseriesResultValue(values)
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Result<TimeseriesResultValue>> getResultTypeReference()
|
||||
{
|
||||
|
@ -262,4 +236,48 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
|
|||
{
|
||||
return Ordering.natural();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> makePreComputeManipulatorFn(
|
||||
final TimeseriesQuery query, final MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
return makeComputeManipulatorFn(query, fn, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> makePostComputeManipulatorFn(
|
||||
TimeseriesQuery query, MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
return makeComputeManipulatorFn(query, fn, true);
|
||||
}
|
||||
|
||||
private Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>> makeComputeManipulatorFn(
|
||||
final TimeseriesQuery query, final MetricManipulationFn fn, final boolean calculatePostAggs
|
||||
)
|
||||
{
|
||||
return new Function<Result<TimeseriesResultValue>, Result<TimeseriesResultValue>>()
|
||||
{
|
||||
@Override
|
||||
public Result<TimeseriesResultValue> apply(Result<TimeseriesResultValue> result)
|
||||
{
|
||||
final Map<String, Object> values = Maps.newHashMap();
|
||||
final TimeseriesResultValue holder = result.getValue();
|
||||
if (calculatePostAggs) {
|
||||
for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
|
||||
values.put(postAgg.getName(), postAgg.compute(holder.getBaseObject()));
|
||||
}
|
||||
}
|
||||
for (AggregatorFactory agg : query.getAggregatorSpecs()) {
|
||||
values.put(agg.getName(), fn.manipulate(agg, holder.getMetric(agg.getName())));
|
||||
}
|
||||
|
||||
return new Result<TimeseriesResultValue>(
|
||||
result.getTimestamp(),
|
||||
new TimeseriesResultValue(values)
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -90,7 +90,6 @@ public class TopNBinaryFn implements BinaryFn<Result<TopNResultValue>, Result<To
|
|||
Map<String, DimensionAndMetricValueExtractor> retVals = new LinkedHashMap<String, DimensionAndMetricValueExtractor>();
|
||||
|
||||
String dimension = dimensionSpec.getOutputName();
|
||||
String topNMetricName = topNMetricSpec.getMetricName(dimensionSpec);
|
||||
for (DimensionAndMetricValueExtractor arg1Val : arg1Vals) {
|
||||
retVals.put(arg1Val.getStringDimensionValue(dimension), arg1Val);
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import io.druid.query.Result;
|
|||
import io.druid.query.ResultGranularTimestampComparator;
|
||||
import io.druid.query.ResultMergeQueryRunner;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.MetricManipulationFn;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
|
@ -139,7 +140,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TopNResultValue>, Result<TopNResultValue>> makeMetricManipulatorFn(
|
||||
public Function<Result<TopNResultValue>, Result<TopNResultValue>> makePreComputeManipulatorFn(
|
||||
final TopNQuery query, final MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
|
@ -148,7 +149,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
private String dimension = query.getDimensionSpec().getOutputName();
|
||||
|
||||
@Override
|
||||
public Result<TopNResultValue> apply(@Nullable Result<TopNResultValue> result)
|
||||
public Result<TopNResultValue> apply(Result<TopNResultValue> result)
|
||||
{
|
||||
List<Map<String, Object>> serializedValues = Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
|
@ -156,13 +157,13 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
new Function<DimensionAndMetricValueExtractor, Map<String, Object>>()
|
||||
{
|
||||
@Override
|
||||
public Map<String, Object> apply(@Nullable DimensionAndMetricValueExtractor input)
|
||||
public Map<String, Object> apply(DimensionAndMetricValueExtractor input)
|
||||
{
|
||||
final Map<String, Object> values = Maps.newHashMap();
|
||||
for (AggregatorFactory agg : query.getAggregatorSpecs()) {
|
||||
values.put(agg.getName(), fn.manipulate(agg, input.getMetric(agg.getName())));
|
||||
}
|
||||
for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
|
||||
for (PostAggregator postAgg : prunePostAggregators(query)) {
|
||||
Object calculatedPostAgg = input.getMetric(postAgg.getName());
|
||||
if (calculatedPostAgg != null) {
|
||||
values.put(postAgg.getName(), calculatedPostAgg);
|
||||
|
@ -186,6 +187,56 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TopNResultValue>, Result<TopNResultValue>> makePostComputeManipulatorFn(
|
||||
final TopNQuery query, final MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
return new Function<Result<TopNResultValue>, Result<TopNResultValue>>()
|
||||
{
|
||||
private String dimension = query.getDimensionSpec().getOutputName();
|
||||
|
||||
@Override
|
||||
public Result<TopNResultValue> apply(Result<TopNResultValue> result)
|
||||
{
|
||||
List<Map<String, Object>> serializedValues = Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
result.getValue(),
|
||||
new Function<DimensionAndMetricValueExtractor, Map<String, Object>>()
|
||||
{
|
||||
@Override
|
||||
public Map<String, Object> apply(DimensionAndMetricValueExtractor input)
|
||||
{
|
||||
final Map<String, Object> values = Maps.newHashMap();
|
||||
// compute all post aggs
|
||||
for (PostAggregator postAgg : query.getPostAggregatorSpecs()) {
|
||||
Object calculatedPostAgg = input.getMetric(postAgg.getName());
|
||||
if (calculatedPostAgg != null) {
|
||||
values.put(postAgg.getName(), calculatedPostAgg);
|
||||
} else {
|
||||
values.put(postAgg.getName(), postAgg.compute(input.getBaseObject()));
|
||||
}
|
||||
}
|
||||
for (AggregatorFactory agg : query.getAggregatorSpecs()) {
|
||||
values.put(agg.getName(), fn.manipulate(agg, input.getMetric(agg.getName())));
|
||||
}
|
||||
|
||||
values.put(dimension, input.getDimensionValue(dimension));
|
||||
|
||||
return values;
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
return new Result<TopNResultValue>(
|
||||
result.getTimestamp(),
|
||||
new TopNResultValue(serializedValues)
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Result<TopNResultValue>> getResultTypeReference()
|
||||
{
|
||||
|
@ -198,7 +249,6 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
return new CacheStrategy<Result<TopNResultValue>, Object, TopNQuery>()
|
||||
{
|
||||
private final List<AggregatorFactory> aggs = query.getAggregatorSpecs();
|
||||
private final List<PostAggregator> postAggs = query.getPostAggregatorSpecs();
|
||||
|
||||
@Override
|
||||
public byte[] computeCacheKey(TopNQuery query)
|
||||
|
@ -238,7 +288,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
return new Function<Result<TopNResultValue>, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object apply(@Nullable final Result<TopNResultValue> input)
|
||||
public Object apply(final Result<TopNResultValue> input)
|
||||
{
|
||||
List<DimensionAndMetricValueExtractor> results = Lists.newArrayList(input.getValue());
|
||||
final List<Object> retVal = Lists.newArrayListWithCapacity(results.size() + 1);
|
||||
|
@ -266,7 +316,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
private final QueryGranularity granularity = query.getGranularity();
|
||||
|
||||
@Override
|
||||
public Result<TopNResultValue> apply(@Nullable Object input)
|
||||
public Result<TopNResultValue> apply(Object input)
|
||||
{
|
||||
List<Object> results = (List<Object>) input;
|
||||
List<Map<String, Object>> retVal = Lists.newArrayListWithCapacity(results.size());
|
||||
|
@ -367,7 +417,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
new Function<Result<TopNResultValue>, Result<TopNResultValue>>()
|
||||
{
|
||||
@Override
|
||||
public Result<TopNResultValue> apply(@Nullable Result<TopNResultValue> input)
|
||||
public Result<TopNResultValue> apply(Result<TopNResultValue> input)
|
||||
{
|
||||
return new Result<TopNResultValue>(
|
||||
input.getTimestamp(),
|
||||
|
@ -405,4 +455,12 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
|
|||
);
|
||||
}
|
||||
}
|
||||
|
||||
private static List<PostAggregator> prunePostAggregators(TopNQuery query)
|
||||
{
|
||||
return AggregatorUtil.pruneDependentPostAgg(
|
||||
query.getPostAggregatorSpecs(),
|
||||
query.getTopNMetricSpec().getMetricName(query.getDimensionSpec())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import io.druid.query.aggregation.AggregatorFactory;
|
|||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||
|
@ -86,6 +87,12 @@ public class QueryRunnerTestHelper
|
|||
)
|
||||
);
|
||||
|
||||
public static final String hyperUniqueFinalizingPostAggMetric = "hyperUniqueFinalizingPostAggMetric";
|
||||
public static ArithmeticPostAggregator hyperUniqueFinalizingPostAgg = new ArithmeticPostAggregator(
|
||||
hyperUniqueFinalizingPostAggMetric,
|
||||
"+",
|
||||
Lists.newArrayList(new HyperUniqueFinalizingPostAggregator(uniqueMetric), new ConstantPostAggregator(null, 1, 1))
|
||||
);
|
||||
|
||||
public static final List<AggregatorFactory> commonAggregators = Arrays.asList(
|
||||
rowsCount,
|
||||
|
|
|
@ -145,4 +145,73 @@ public class AggregatorUtilTest
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullPostAggregatorNames()
|
||||
{
|
||||
AggregatorFactory agg1 = new DoubleSumAggregatorFactory("agg1", "value");
|
||||
AggregatorFactory agg2 = new DoubleSumAggregatorFactory("agg2", "count");
|
||||
PostAggregator postAgg1 = new ArithmeticPostAggregator(
|
||||
null, "*", Lists.<PostAggregator>newArrayList(
|
||||
new FieldAccessPostAggregator(
|
||||
null,
|
||||
"agg1"
|
||||
), new FieldAccessPostAggregator(null, "agg2")
|
||||
)
|
||||
);
|
||||
|
||||
PostAggregator postAgg2 = new ArithmeticPostAggregator(
|
||||
"postAgg",
|
||||
"/",
|
||||
Lists.<PostAggregator>newArrayList(
|
||||
new FieldAccessPostAggregator(
|
||||
null,
|
||||
"agg1"
|
||||
), new FieldAccessPostAggregator(null, "agg2")
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
new Pair(Lists.newArrayList(agg1, agg2), Lists.newArrayList(postAgg2)), AggregatorUtil.condensedAggregators(
|
||||
Lists.newArrayList(agg1, agg2),
|
||||
Lists.newArrayList(postAgg1, postAgg2),
|
||||
"postAgg"
|
||||
)
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCasing()
|
||||
{
|
||||
AggregatorFactory agg1 = new DoubleSumAggregatorFactory("Agg1", "value");
|
||||
AggregatorFactory agg2 = new DoubleSumAggregatorFactory("Agg2", "count");
|
||||
PostAggregator postAgg1 = new ArithmeticPostAggregator(
|
||||
null, "*", Lists.<PostAggregator>newArrayList(
|
||||
new FieldAccessPostAggregator(
|
||||
null,
|
||||
"Agg1"
|
||||
), new FieldAccessPostAggregator(null, "Agg2")
|
||||
)
|
||||
);
|
||||
|
||||
PostAggregator postAgg2 = new ArithmeticPostAggregator(
|
||||
"postAgg",
|
||||
"/",
|
||||
Lists.<PostAggregator>newArrayList(
|
||||
new FieldAccessPostAggregator(
|
||||
null,
|
||||
"Agg1"
|
||||
), new FieldAccessPostAggregator(null, "Agg2")
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
new Pair(Lists.newArrayList(agg1, agg2), Lists.newArrayList(postAgg2)), AggregatorUtil.condensedAggregators(
|
||||
Lists.newArrayList(agg1, agg2),
|
||||
Lists.newArrayList(postAgg1, postAgg2),
|
||||
"postAgg"
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1200,7 +1200,8 @@ public class TopNQueryRunnerTest
|
|||
.postAggregators(
|
||||
Arrays.<PostAggregator>asList(
|
||||
QueryRunnerTestHelper.addRowsIndexConstant,
|
||||
QueryRunnerTestHelper.dependentPostAgg
|
||||
QueryRunnerTestHelper.dependentPostAgg,
|
||||
QueryRunnerTestHelper.hyperUniqueFinalizingPostAgg
|
||||
)
|
||||
)
|
||||
.build();
|
||||
|
@ -1219,6 +1220,10 @@ public class TopNQueryRunnerTest
|
|||
.put("uniques", QueryRunnerTestHelper.UNIQUES_2)
|
||||
.put("maxIndex", 1743.9217529296875D)
|
||||
.put("minIndex", 792.3260498046875D)
|
||||
.put(
|
||||
QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
|
||||
QueryRunnerTestHelper.UNIQUES_2 + 1.0
|
||||
)
|
||||
.build(),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put(providerDimension, "upfront")
|
||||
|
@ -1229,6 +1234,10 @@ public class TopNQueryRunnerTest
|
|||
.put("uniques", QueryRunnerTestHelper.UNIQUES_2)
|
||||
.put("maxIndex", 1870.06103515625D)
|
||||
.put("minIndex", 545.9906005859375D)
|
||||
.put(
|
||||
QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
|
||||
QueryRunnerTestHelper.UNIQUES_2 + 1.0
|
||||
)
|
||||
.build(),
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put(providerDimension, "spot")
|
||||
|
@ -1237,6 +1246,10 @@ public class TopNQueryRunnerTest
|
|||
.put("addRowsIndexConstant", 96444.57232284546D)
|
||||
.put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D)
|
||||
.put("uniques", QueryRunnerTestHelper.UNIQUES_9)
|
||||
.put(
|
||||
QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric,
|
||||
QueryRunnerTestHelper.UNIQUES_9 + 1.0
|
||||
)
|
||||
.put("maxIndex", 277.2735290527344D)
|
||||
.put("minIndex", 59.02102279663086D)
|
||||
.build()
|
||||
|
|
|
@ -357,7 +357,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
}
|
||||
}
|
||||
),
|
||||
toolChest.makeMetricManipulatorFn(
|
||||
toolChest.makePreComputeManipulatorFn(
|
||||
rewrittenQuery,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
|
|
|
@ -122,8 +122,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
final JavaType typeRef;
|
||||
if (isBySegment) {
|
||||
typeRef = types.rhs;
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
typeRef = types.lhs;
|
||||
}
|
||||
|
||||
|
@ -218,15 +217,16 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
if (!isBySegment) {
|
||||
retVal = Sequences.map(
|
||||
retVal,
|
||||
toolChest.makeMetricManipulatorFn(
|
||||
query, new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.deserialize(object);
|
||||
}
|
||||
}
|
||||
toolChest.makePreComputeManipulatorFn(
|
||||
query,
|
||||
new MetricManipulationFn()
|
||||
{
|
||||
@Override
|
||||
public Object manipulate(AggregatorFactory factory, Object object)
|
||||
{
|
||||
return factory.deserialize(object);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -313,7 +313,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
if(jp != null) {
|
||||
if (jp != null) {
|
||||
jp.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -116,7 +116,8 @@ import java.util.concurrent.Executor;
|
|||
@RunWith(Parameterized.class)
|
||||
public class CachingClusteredClientTest
|
||||
{
|
||||
public static final ImmutableMap<String, Object> CONTEXT = ImmutableMap.of();
|
||||
public static final ImmutableMap<String, Object> CONTEXT = ImmutableMap.<String, Object>of("finalize", false);
|
||||
|
||||
public static final MultipleIntervalSegmentSpec SEG_SPEC = new MultipleIntervalSegmentSpec(ImmutableList.<Interval>of());
|
||||
public static final String DATA_SOURCE = "test";
|
||||
protected static final DefaultObjectMapper jsonMapper = new DefaultObjectMapper(new SmileFactory());
|
||||
|
@ -321,10 +322,11 @@ public class CachingClusteredClientTest
|
|||
.filters(DIM_FILTER)
|
||||
.granularity(GRANULARITY)
|
||||
.aggregators(AGGS)
|
||||
.postAggregators(POST_AGGS);
|
||||
|
||||
.postAggregators(POST_AGGS)
|
||||
.context(CONTEXT);
|
||||
QueryRunner runner = new FinalizeResultsQueryRunner(client, new TimeseriesQueryQueryToolChest(new QueryConfig()));
|
||||
testQueryCaching(
|
||||
client,
|
||||
runner,
|
||||
1,
|
||||
true,
|
||||
builder.context(
|
||||
|
@ -343,7 +345,7 @@ public class CachingClusteredClientTest
|
|||
cache.close("0_0");
|
||||
|
||||
testQueryCaching(
|
||||
client,
|
||||
runner,
|
||||
1,
|
||||
false,
|
||||
builder.context(
|
||||
|
|
|
@ -578,7 +578,7 @@ public class ServerManagerTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public Function<T, T> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn)
|
||||
public Function<T, T> makePreComputeManipulatorFn(QueryType query, MetricManipulationFn fn)
|
||||
{
|
||||
return Functions.identity();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue