mirror of https://github.com/apache/druid.git
Merge pull request #1044 from metamx/groupby-postagg-naming-bug
Ensure unique names for aggregators / post-aggregators + optimize groupBy post-aggregators
This commit is contained in:
commit
c97fbdf616
|
@ -214,4 +214,64 @@ public class ApproximateHistogramGroupByQueryTest
|
|||
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo");
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testGroupByWithSameNameComplexPostAgg()
|
||||
{
|
||||
ApproximateHistogramAggregatorFactory aggFactory = new ApproximateHistogramAggregatorFactory(
|
||||
"quantile",
|
||||
"index",
|
||||
10,
|
||||
5,
|
||||
Float.NEGATIVE_INFINITY,
|
||||
Float.POSITIVE_INFINITY
|
||||
);
|
||||
|
||||
GroupByQuery query = new GroupByQuery.Builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setGranularity(QueryRunnerTestHelper.allGran)
|
||||
.setDimensions(
|
||||
Arrays.<DimensionSpec>asList(
|
||||
new DefaultDimensionSpec(
|
||||
QueryRunnerTestHelper.marketDimension,
|
||||
"marketalias"
|
||||
)
|
||||
)
|
||||
)
|
||||
.setInterval(QueryRunnerTestHelper.fullOnInterval)
|
||||
.setLimitSpec(
|
||||
new DefaultLimitSpec(
|
||||
Lists.newArrayList(
|
||||
new OrderByColumnSpec(
|
||||
"marketalias",
|
||||
OrderByColumnSpec.Direction.DESCENDING
|
||||
)
|
||||
), 1
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(
|
||||
Lists.newArrayList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
aggFactory
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new QuantilePostAggregator("quantile", "quantile", 0.5f)
|
||||
)
|
||||
)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"1970-01-01T00:00:00.000Z",
|
||||
"marketalias", "upfront",
|
||||
"rows", 186L,
|
||||
"quantile", 880.9881f
|
||||
)
|
||||
);
|
||||
|
||||
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,9 +17,7 @@
|
|||
|
||||
package io.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -39,30 +37,23 @@ public class Queries
|
|||
Preconditions.checkNotNull(aggFactories, "aggregations cannot be null");
|
||||
Preconditions.checkArgument(aggFactories.size() > 0, "Must have at least one AggregatorFactory");
|
||||
|
||||
final Set<String> aggNames = Sets.newHashSet();
|
||||
for (AggregatorFactory aggFactory : aggFactories) {
|
||||
Preconditions.checkArgument(aggNames.add(aggFactory.getName()), "[%s] already defined", aggFactory.getName());
|
||||
}
|
||||
|
||||
if (postAggs != null && !postAggs.isEmpty()) {
|
||||
Set<String> combinedAggNames = Sets.newHashSet(
|
||||
Lists.transform(
|
||||
aggFactories,
|
||||
new Function<AggregatorFactory, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(AggregatorFactory input)
|
||||
{
|
||||
return input.getName();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
final Set<String> combinedAggNames = Sets.newHashSet(aggNames);
|
||||
|
||||
for (PostAggregator postAgg : postAggs) {
|
||||
Set<String> dependencies = postAgg.getDependentFields();
|
||||
Set<String> missing = Sets.difference(dependencies, combinedAggNames);
|
||||
final Set<String> dependencies = postAgg.getDependentFields();
|
||||
final Set<String> missing = Sets.difference(dependencies, combinedAggNames);
|
||||
|
||||
Preconditions.checkArgument(
|
||||
missing.isEmpty(),
|
||||
"Missing fields [%s] for postAggregator [%s]", missing, postAgg.getName()
|
||||
);
|
||||
combinedAggNames.add(postAgg.getName());
|
||||
Preconditions.checkArgument(combinedAggNames.add(postAgg.getName()), "[%s] already defined");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,10 +21,13 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.Collections2;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
|
@ -59,10 +62,12 @@ import io.druid.segment.incremental.IncrementalIndex;
|
|||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -115,15 +120,21 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
}
|
||||
|
||||
if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) {
|
||||
return mergeGroupByResults(((GroupByQuery) input), runner, responseContext
|
||||
);
|
||||
return mergeGroupByResults(
|
||||
(GroupByQuery) input,
|
||||
runner,
|
||||
responseContext);
|
||||
}
|
||||
return runner.run(input, responseContext);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Sequence<Row> mergeGroupByResults(final GroupByQuery query, QueryRunner<Row> runner, Map<String, Object> context)
|
||||
private Sequence<Row> mergeGroupByResults(
|
||||
final GroupByQuery query,
|
||||
QueryRunner<Row> runner,
|
||||
Map<String, Object> context
|
||||
)
|
||||
{
|
||||
// If there's a subquery, merge subquery results and then apply the aggregator
|
||||
|
||||
|
@ -168,7 +179,28 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
|
|||
index
|
||||
);
|
||||
} else {
|
||||
final IncrementalIndex index = makeIncrementalIndex(query, runner.run(query, context));
|
||||
final IncrementalIndex index = makeIncrementalIndex(
|
||||
query, runner.run(
|
||||
new GroupByQuery(
|
||||
query.getDataSource(),
|
||||
query.getQuerySegmentSpec(),
|
||||
query.getDimFilter(),
|
||||
query.getGranularity(),
|
||||
query.getDimensions(),
|
||||
query.getAggregatorSpecs(),
|
||||
// Don't do post aggs until the end of this method.
|
||||
ImmutableList.<PostAggregator>of(),
|
||||
query.getHavingSpec(),
|
||||
query.getLimitSpec(),
|
||||
query.getContext()
|
||||
).withOverriddenContext(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"finalize", false
|
||||
)
|
||||
)
|
||||
, context
|
||||
)
|
||||
);
|
||||
return new ResourceClosingSequence<>(query.applyLimit(postAggregate(query, index)), index);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
|||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.MaxAggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
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;
|
||||
|
@ -250,6 +251,44 @@ public class GroupByQueryRunnerTest
|
|||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testGroupByWithUniquesAndPostAggWithSameName()
|
||||
{
|
||||
GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new HyperUniquesAggregatorFactory(
|
||||
"quality_uniques",
|
||||
"quality_uniques"
|
||||
)
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new HyperUniqueFinalizingPostAggregator("quality_uniques")
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.allGran)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-01",
|
||||
"rows",
|
||||
26L,
|
||||
"quality_uniques",
|
||||
QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
);
|
||||
|
||||
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithCardinality()
|
||||
{
|
||||
|
@ -1151,6 +1190,115 @@ public class GroupByQueryRunnerTest
|
|||
TestHelper.assertExpectedObjects(expectedResults, results, "order-limit");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostAggMergedHavingSpec()
|
||||
{
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "index", 4420L, QueryRunnerTestHelper.addRowsIndexConstantMetric,(double) (6l+4420l+1l)),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "index", 4416L, QueryRunnerTestHelper.addRowsIndexConstantMetric, (double) (6l+4416l+1l))
|
||||
);
|
||||
|
||||
GroupByQuery.Builder builder = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("index", "index")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(ImmutableList.<PostAggregator>of(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||
.setHavingSpec(
|
||||
new OrHavingSpec(
|
||||
ImmutableList.<HavingSpec>of(
|
||||
new GreaterThanHavingSpec(QueryRunnerTestHelper.addRowsIndexConstantMetric, 1000L)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
final GroupByQuery fullQuery = builder.build();
|
||||
|
||||
QueryRunner mergedRunner = factory.getToolchest().mergeResults(
|
||||
new QueryRunner<Row>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Row> run(
|
||||
Query<Row> query, Map<String, Object> responseContext
|
||||
)
|
||||
{
|
||||
// simulate two daily segments
|
||||
final Query query1 = query.withQuerySegmentSpec(
|
||||
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03")))
|
||||
);
|
||||
final Query query2 = query.withQuerySegmentSpec(
|
||||
new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04")))
|
||||
);
|
||||
return Sequences.concat(runner.run(query1, responseContext), runner.run(query2, responseContext));
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
Map<String, Object> context = Maps.newHashMap();
|
||||
TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery, context), "merged");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPostAggHavingSpec()
|
||||
{
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-01",
|
||||
"alias",
|
||||
"mezzanine",
|
||||
"rows",
|
||||
6L,
|
||||
"index",
|
||||
4420L,
|
||||
QueryRunnerTestHelper.addRowsIndexConstantMetric,
|
||||
(double) (6l + 4420l + 1l)
|
||||
),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-01",
|
||||
"alias",
|
||||
"premium",
|
||||
"rows",
|
||||
6L,
|
||||
"index",
|
||||
4416L,
|
||||
QueryRunnerTestHelper.addRowsIndexConstantMetric,
|
||||
(double) (6l + 4416l + 1l)
|
||||
)
|
||||
);
|
||||
|
||||
GroupByQuery.Builder builder = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("index", "index")
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(ImmutableList.<PostAggregator>of(QueryRunnerTestHelper.addRowsIndexConstant))
|
||||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||
.setHavingSpec(
|
||||
new OrHavingSpec(
|
||||
ImmutableList.<HavingSpec>of(
|
||||
new GreaterThanHavingSpec(QueryRunnerTestHelper.addRowsIndexConstantMetric, 1000L)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
final GroupByQuery fullQuery = builder.build();
|
||||
TestHelper.assertExpectedObjects(expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery), "");
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testHavingSpec()
|
||||
{
|
||||
|
@ -1160,6 +1308,40 @@ public class GroupByQueryRunnerTest
|
|||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L)
|
||||
);
|
||||
|
||||
GroupByQuery.Builder builder = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval("2011-04-02/2011-04-04")
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
)
|
||||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null))
|
||||
.setHavingSpec(
|
||||
new OrHavingSpec(
|
||||
ImmutableList.of(
|
||||
new GreaterThanHavingSpec("rows", 2L),
|
||||
new EqualToHavingSpec("idx", 217L)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
final GroupByQuery fullQuery = builder.build();
|
||||
TestHelper.assertExpectedObjects(expectedResults, GroupByQueryRunnerTestHelper.runQuery(factory, runner, fullQuery), "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergedHavingSpec()
|
||||
{
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 2L, "idx", 217L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 6L, "idx", 4416L)
|
||||
);
|
||||
|
||||
GroupByQuery.Builder builder = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
|
@ -1577,7 +1759,7 @@ public class GroupByQueryRunnerTest
|
|||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx", "+", Arrays.asList(
|
||||
"idx_post", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_agg", "idx"),
|
||||
new ConstantPostAggregator("ten_thousand", 10000)
|
||||
)
|
||||
|
@ -1589,41 +1771,45 @@ public class GroupByQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx_post", 11135.0, "idx", 1135L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx_post", 11118.0, "idx", 1118L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-01",
|
||||
"alias",
|
||||
"entertainment",
|
||||
"rows",
|
||||
1L,
|
||||
"idx_post",
|
||||
11158.0,
|
||||
"idx",
|
||||
11158.0
|
||||
1158L
|
||||
),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx_post", 11120.0, "idx", 1120L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx_post", 13870.0, "idx", 3870L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx_post", 11121.0, "idx", 1121L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx_post", 13900.0, "idx", 3900L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx_post", 11078.0, "idx", 1078L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx_post", 11119.0, "idx", 1119L),
|
||||
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx_post", 11147.0, "idx", 1147L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx_post", 11112.0, "idx", 1112L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-02",
|
||||
"alias",
|
||||
"entertainment",
|
||||
"rows",
|
||||
1L,
|
||||
"idx_post",
|
||||
11166.0,
|
||||
"idx",
|
||||
11166.0
|
||||
1166L
|
||||
),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0)
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx_post", 11113.0, "idx", 1113L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx_post", 13447.0, "idx", 3447L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx_post", 11114.0, "idx", 1114L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx_post", 13505.0, "idx", 3505L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx_post", 11097.0, "idx", 1097L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx_post", 11126.0, "idx", 1126L)
|
||||
);
|
||||
|
||||
// Subqueries are handled by the ToolChest
|
||||
|
@ -1693,7 +1879,7 @@ public class GroupByQueryRunnerTest
|
|||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx", "+", Arrays.asList(
|
||||
"idx_post", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_agg", "idx"),
|
||||
new ConstantPostAggregator("ten_thousand", 10000)
|
||||
)
|
||||
|
@ -1705,39 +1891,43 @@ public class GroupByQueryRunnerTest
|
|||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx_post", 11135.0, "idx", 1135L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx_post", 11118.0, "idx", 1118L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-01",
|
||||
"alias",
|
||||
"entertainment",
|
||||
"rows",
|
||||
1L,
|
||||
"idx_post",
|
||||
11158.0,
|
||||
"idx",
|
||||
11158.0
|
||||
1158L
|
||||
),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx_post", 11120.0, "idx", 1120L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx_post", 11121.0, "idx", 1121L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx_post", 11078.0, "idx", 1078L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx_post", 11119.0, "idx", 1119L),
|
||||
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx_post", 11147.0, "idx", 1147L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx_post", 11112.0, "idx", 1112L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||
"2011-04-02",
|
||||
"alias",
|
||||
"entertainment",
|
||||
"rows",
|
||||
1L,
|
||||
"idx_post",
|
||||
11166.0,
|
||||
"idx",
|
||||
11166.0
|
||||
1166L
|
||||
),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0)
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx_post", 11113.0, "idx", 1113L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx_post", 13447.0, "idx", 3447L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx_post", 11114.0, "idx", 1114L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx_post", 13505.0, "idx", 3505L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx_post", 11097.0, "idx", 1097L),
|
||||
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx_post", 11126.0, "idx", 1126L)
|
||||
);
|
||||
|
||||
// Subqueries are handled by the ToolChest
|
||||
|
@ -1815,7 +2005,7 @@ public class GroupByQueryRunnerTest
|
|||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new ArithmeticPostAggregator(
|
||||
"idx", "+", Arrays.asList(
|
||||
"idx_post", "+", Arrays.asList(
|
||||
new FieldAccessPostAggregator("the_idx_agg", "idx"),
|
||||
new ConstantPostAggregator("ten_thousand", 10000)
|
||||
)
|
||||
|
@ -1844,8 +2034,10 @@ public class GroupByQueryRunnerTest
|
|||
"travel",
|
||||
"rows",
|
||||
1L,
|
||||
"idx",
|
||||
"idx_post",
|
||||
11119.0,
|
||||
"idx",
|
||||
1119L,
|
||||
"js_outer_agg",
|
||||
123.92274475097656
|
||||
),
|
||||
|
@ -1855,8 +2047,10 @@ public class GroupByQueryRunnerTest
|
|||
"technology",
|
||||
"rows",
|
||||
1L,
|
||||
"idx",
|
||||
"idx_post",
|
||||
11078.0,
|
||||
"idx",
|
||||
1078L,
|
||||
"js_outer_agg",
|
||||
82.62254333496094
|
||||
),
|
||||
|
@ -1866,8 +2060,10 @@ public class GroupByQueryRunnerTest
|
|||
"news",
|
||||
"rows",
|
||||
1L,
|
||||
"idx",
|
||||
"idx_post",
|
||||
11121.0,
|
||||
"idx",
|
||||
1121L,
|
||||
"js_outer_agg",
|
||||
125.58358001708984
|
||||
),
|
||||
|
@ -1877,8 +2073,10 @@ public class GroupByQueryRunnerTest
|
|||
"health",
|
||||
"rows",
|
||||
1L,
|
||||
"idx",
|
||||
"idx_post",
|
||||
11120.0,
|
||||
"idx",
|
||||
1120L,
|
||||
"js_outer_agg",
|
||||
124.13470458984375
|
||||
),
|
||||
|
@ -1888,8 +2086,10 @@ public class GroupByQueryRunnerTest
|
|||
"entertainment",
|
||||
"rows",
|
||||
1L,
|
||||
"idx",
|
||||
"idx_post",
|
||||
11158.0,
|
||||
"idx",
|
||||
1158L,
|
||||
"js_outer_agg",
|
||||
162.74722290039062
|
||||
)
|
||||
|
|
Loading…
Reference in New Issue