Merge pull request #1799 from dclim/nested-groupby-aggregator-fix

Support multiple outer aggregators of same type and provide more help…
This commit is contained in:
Fangjin Yang 2015-10-29 18:01:31 -07:00
commit 25a0eb7ed5
2 changed files with 80 additions and 3 deletions

View File

@ -30,7 +30,9 @@ import com.google.common.collect.Iterables;
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.IAE;
import com.metamx.common.ISE;
import com.metamx.common.Pair;
import com.metamx.common.guava.Accumulator;
@ -160,14 +162,34 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery
}
final Sequence<Row> subqueryResult = mergeGroupByResults(subquery, runner, context);
final List<AggregatorFactory> aggs = Lists.newArrayList();
final Set<AggregatorFactory> aggs = Sets.newHashSet();
// Nested group-bys work by first running the inner query and then materializing the results in an incremental
// index which the outer query is then run against. To build the incremental index, we use the fieldNames from
// the aggregators for the outer query to define the column names so that the index will match the query. If
// there are multiple types of aggregators in the outer query referencing the same fieldName, we will try to build
// multiple columns of the same name using different aggregator types and will fail. Here, we permit multiple
// aggregators of the same type referencing the same fieldName (and skip creating identical columns for the
// subsequent ones) and return an error if the aggregator types are different.
for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {
aggs.addAll(aggregatorFactory.getRequiredColumns());
for (final AggregatorFactory transferAgg : aggregatorFactory.getRequiredColumns()) {
if (Iterables.any(aggs, new Predicate<AggregatorFactory>() {
@Override
public boolean apply(AggregatorFactory agg) {
return agg.getName().equals(transferAgg.getName()) && !agg.equals(transferAgg);
}
})) {
throw new IAE("Inner aggregator can currently only be referenced by a single type of outer aggregator" +
" for '%s'", transferAgg.getName());
}
aggs.add(transferAgg);
}
}
// We need the inner incremental index to have all the columns required by the outer query
final GroupByQuery innerQuery = new GroupByQuery.Builder(subquery)
.setAggregatorSpecs(aggs)
.setAggregatorSpecs(Lists.newArrayList(aggs))
.setInterval(subquery.getIntervals())
.setPostAggregatorSpecs(Lists.<PostAggregator>newArrayList())
.build();

View File

@ -2502,6 +2502,61 @@ public class GroupByQueryRunnerTest
TestHelper.assertExpectedObjects(expectedResults, results, "");
}
@Test
public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField()
{
GroupByQuery subquery = GroupByQuery
.builder()
.setDataSource(QueryRunnerTestHelper.dataSource)
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
.setAggregatorSpecs(
Arrays.asList(
QueryRunnerTestHelper.rowsCount,
new LongSumAggregatorFactory("idx", "index")
)
)
.setPostAggregatorSpecs(
Lists.<PostAggregator>newArrayList(
new ArithmeticPostAggregator(
"post_agg",
"+",
Lists.<PostAggregator>newArrayList(
new FieldAccessPostAggregator("idx", "idx"),
new FieldAccessPostAggregator("idx", "idx")
)
)
)
)
.setGranularity(QueryRunnerTestHelper.dayGran)
.build();
GroupByQuery query = GroupByQuery
.builder()
.setDataSource(subquery)
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
.setAggregatorSpecs(
Arrays.<AggregatorFactory>asList(
new DoubleMaxAggregatorFactory("idx1", "idx"),
new DoubleMaxAggregatorFactory("idx2", "idx"),
new DoubleMaxAggregatorFactory("idx3", "post_agg"),
new DoubleMaxAggregatorFactory("idx4", "post_agg")
)
)
.setGranularity(QueryRunnerTestHelper.dayGran)
.build();
List<Row> expectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "idx1", 2900.0, "idx2", 2900.0,
"idx3", 5800.0, "idx4", 5800.0),
GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx1", 2505.0, "idx2", 2505.0,
"idx3", 5010.0, "idx4", 5010.0)
);
Iterable<Row> results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
TestHelper.assertExpectedObjects(expectedResults, results, "");
}
@Test
public void testDifferentGroupingSubqueryWithFilter()