From 7c21d95f48c372ceb266e85639c973a54a778d99 Mon Sep 17 00:00:00 2001 From: "Xiaotian (Jackie) Jiang" Date: Mon, 16 Dec 2024 00:28:49 -0800 Subject: [PATCH 1/6] Support is_enable_group_trim agg option --- pinot-common/src/main/proto/plan.proto | 2 + .../calcite/rel/hint/PinotHintOptions.java | 1 + .../rel/logical/PinotLogicalAggregate.java | 55 +++-- .../PinotAggregateExchangeNodeInsertRule.java | 202 +++++++++++++----- .../calcite/rel/rules/PinotQueryRuleSets.java | 4 +- .../parser/CalciteRexExpressionParser.java | 4 +- .../query/planner/explain/PlanNodeMerger.java | 6 + .../logical/EquivalentStagesFinder.java | 4 +- .../logical/RelToPlanNodeConverter.java | 2 +- .../query/planner/plannode/AggregateNode.java | 29 ++- .../planner/serde/PlanNodeDeserializer.java | 3 +- .../planner/serde/PlanNodeSerializer.java | 2 + .../test/resources/queries/GroupByPlans.json | 33 +++ .../plan/server/ServerPlanRequestVisitor.java | 39 ++-- .../operator/AggregateOperatorTest.java | 2 +- .../operator/MultiStageAccountingTest.java | 2 +- .../test/resources/queries/QueryHints.json | 4 + 17 files changed, 289 insertions(+), 105 deletions(-) diff --git a/pinot-common/src/main/proto/plan.proto b/pinot-common/src/main/proto/plan.proto index 49d357307648..e3b2bbf65482 100644 --- a/pinot-common/src/main/proto/plan.proto +++ b/pinot-common/src/main/proto/plan.proto @@ -69,6 +69,8 @@ message AggregateNode { repeated int32 groupKeys = 3; AggType aggType = 4; bool leafReturnFinalResult = 5; + repeated Collation collations = 6; + int32 limit = 7; } message FilterNode { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java index 558b2f898539..82f80da4bee2 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java @@ -43,6 +43,7 @@ public static class AggregateOptions { public static final String IS_PARTITIONED_BY_GROUP_BY_KEYS = "is_partitioned_by_group_by_keys"; public static final String IS_LEAF_RETURN_FINAL_RESULT = "is_leaf_return_final_result"; public static final String SKIP_LEAF_STAGE_GROUP_BY_AGGREGATION = "is_skip_leaf_stage_group_by"; + public static final String ENABLE_GROUP_TRIM = "is_enable_group_trim"; public static final String NUM_GROUPS_LIMIT = "num_groups_limit"; public static final String MAX_INITIAL_RESULT_HOLDER_CAPACITY = "max_initial_result_holder_capacity"; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalAggregate.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalAggregate.java index 241c44703e6b..b382f9e63598 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalAggregate.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalAggregate.java @@ -22,6 +22,7 @@ import javax.annotation.Nullable; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelFieldCollation; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelWriter; import org.apache.calcite.rel.core.Aggregate; @@ -35,39 +36,36 @@ public class PinotLogicalAggregate extends Aggregate { private final AggType _aggType; private final boolean _leafReturnFinalResult; + // The following fields are for group trimming purpose, and are extracted from the Sort on top of this Aggregate. + private final List _collations; + private final int _limit; + public PinotLogicalAggregate(RelOptCluster cluster, RelTraitSet traitSet, List hints, RelNode input, ImmutableBitSet groupSet, @Nullable List groupSets, List aggCalls, - AggType aggType, boolean leafReturnFinalResult) { + AggType aggType, boolean leafReturnFinalResult, @Nullable List collations, int limit) { super(cluster, traitSet, hints, input, groupSet, groupSets, aggCalls); _aggType = aggType; _leafReturnFinalResult = leafReturnFinalResult; + _collations = collations; + _limit = limit; } - public PinotLogicalAggregate(RelOptCluster cluster, RelTraitSet traitSet, List hints, RelNode input, - ImmutableBitSet groupSet, @Nullable List groupSets, List aggCalls, - AggType aggType) { - this(cluster, traitSet, hints, input, groupSet, groupSets, aggCalls, aggType, false); - } - - public PinotLogicalAggregate(Aggregate aggRel, List aggCalls, AggType aggType, - boolean leafReturnFinalResult) { - this(aggRel.getCluster(), aggRel.getTraitSet(), aggRel.getHints(), aggRel.getInput(), aggRel.getGroupSet(), - aggRel.getGroupSets(), aggCalls, aggType, leafReturnFinalResult); + public PinotLogicalAggregate(Aggregate aggRel, RelNode input, ImmutableBitSet groupSet, + @Nullable List groupSets, List aggCalls, AggType aggType, + boolean leafReturnFinalResult, @Nullable List collations, int limit) { + this(aggRel.getCluster(), aggRel.getTraitSet(), aggRel.getHints(), input, groupSet, groupSets, aggCalls, aggType, + leafReturnFinalResult, collations, limit); } - public PinotLogicalAggregate(Aggregate aggRel, List aggCalls, AggType aggType) { - this(aggRel, aggCalls, aggType, false); - } - - public PinotLogicalAggregate(Aggregate aggRel, RelNode input, List aggCalls, AggType aggType) { - this(aggRel.getCluster(), aggRel.getTraitSet(), aggRel.getHints(), input, aggRel.getGroupSet(), - aggRel.getGroupSets(), aggCalls, aggType); + public PinotLogicalAggregate(Aggregate aggRel, RelNode input, List aggCalls, AggType aggType, + boolean leafReturnFinalResult, @Nullable List collations, int limit) { + this(aggRel, input, aggRel.getGroupSet(), aggRel.getGroupSets(), aggCalls, aggType, + leafReturnFinalResult, collations, limit); } public PinotLogicalAggregate(Aggregate aggRel, RelNode input, ImmutableBitSet groupSet, List aggCalls, - AggType aggType, boolean leafReturnFinalResult) { - this(aggRel.getCluster(), aggRel.getTraitSet(), aggRel.getHints(), input, groupSet, null, aggCalls, aggType, - leafReturnFinalResult); + AggType aggType, boolean leafReturnFinalResult, @Nullable List collations, int limit) { + this(aggRel, input, groupSet, null, aggCalls, aggType, leafReturnFinalResult, collations, limit); } public AggType getAggType() { @@ -78,11 +76,20 @@ public boolean isLeafReturnFinalResult() { return _leafReturnFinalResult; } + @Nullable + public List getCollations() { + return _collations; + } + + public int getLimit() { + return _limit; + } + @Override public PinotLogicalAggregate copy(RelTraitSet traitSet, RelNode input, ImmutableBitSet groupSet, @Nullable List groupSets, List aggCalls) { return new PinotLogicalAggregate(getCluster(), traitSet, hints, input, groupSet, groupSets, aggCalls, _aggType, - _leafReturnFinalResult); + _leafReturnFinalResult, _collations, _limit); } @Override @@ -90,12 +97,14 @@ public RelWriter explainTerms(RelWriter pw) { RelWriter relWriter = super.explainTerms(pw); relWriter.item("aggType", _aggType); relWriter.itemIf("leafReturnFinalResult", true, _leafReturnFinalResult); + relWriter.itemIf("collations", _collations, _collations != null); + relWriter.itemIf("limit", _limit, _limit > 0); return relWriter; } @Override public RelNode withHints(List hintList) { return new PinotLogicalAggregate(getCluster(), traitSet, hintList, input, groupSet, groupSets, aggCalls, _aggType, - _leafReturnFinalResult); + _leafReturnFinalResult, _collations, _limit); } } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java index df11fdb49a2e..561dcc9c281c 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotAggregateExchangeNodeInsertRule.java @@ -28,10 +28,12 @@ import org.apache.calcite.rel.RelCollation; import org.apache.calcite.rel.RelDistribution; import org.apache.calcite.rel.RelDistributions; +import org.apache.calcite.rel.RelFieldCollation; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.Sort; import org.apache.calcite.rel.core.Union; import org.apache.calcite.rel.logical.LogicalAggregate; import org.apache.calcite.rel.rules.AggregateExtractProjectRule; @@ -83,48 +85,148 @@ * - COUNT(*)__LEAF produces TUPLE[ SUM(1), GROUP_BY_KEY ] * - COUNT(*)__FINAL produces TUPLE[ SUM(COUNT(*)__LEAF), GROUP_BY_KEY ] */ -public class PinotAggregateExchangeNodeInsertRule extends RelOptRule { - public static final PinotAggregateExchangeNodeInsertRule INSTANCE = - new PinotAggregateExchangeNodeInsertRule(PinotRuleUtils.PINOT_REL_FACTORY); - - public PinotAggregateExchangeNodeInsertRule(RelBuilderFactory factory) { - // NOTE: Explicitly match for LogicalAggregate because after applying the rule, LogicalAggregate is replaced with - // PinotLogicalAggregate, and the rule won't be applied again. - super(operand(LogicalAggregate.class, any()), factory, null); +public class PinotAggregateExchangeNodeInsertRule { + + public static class SortProjectAggregate extends RelOptRule { + public static final SortProjectAggregate INSTANCE = new SortProjectAggregate(PinotRuleUtils.PINOT_REL_FACTORY); + + private SortProjectAggregate(RelBuilderFactory factory) { + // NOTE: Explicitly match for LogicalAggregate because after applying the rule, LogicalAggregate is replaced with + // PinotLogicalAggregate, and the rule won't be applied again. + super(operand(Sort.class, operand(Project.class, operand(LogicalAggregate.class, any()))), factory, null); + } + + @Override + public void onMatch(RelOptRuleCall call) { + // Apply this rule for group-by queries with enable group trim hint. + LogicalAggregate aggRel = call.rel(2); + if (aggRel.getGroupSet().isEmpty()) { + return; + } + Map hintOptions = + PinotHintStrategyTable.getHintOptions(aggRel.getHints(), PinotHintOptions.AGGREGATE_HINT_OPTIONS); + if (hintOptions == null || !Boolean.parseBoolean( + hintOptions.get(PinotHintOptions.AggregateOptions.ENABLE_GROUP_TRIM))) { + return; + } + + Sort sortRel = call.rel(0); + Project projectRel = call.rel(1); + List projects = projectRel.getProjects(); + List collations = sortRel.getCollation().getFieldCollations(); + if (collations.isEmpty()) { + // Cannot enable group trim without sort key. + return; + } + List newCollations = new ArrayList<>(collations.size()); + for (RelFieldCollation fieldCollation : collations) { + RexNode project = projects.get(fieldCollation.getFieldIndex()); + if (project instanceof RexInputRef) { + newCollations.add(fieldCollation.withFieldIndex(((RexInputRef) project).getIndex())); + } else { + // Cannot enable group trim when the sort key is not a direct reference to the input. + return; + } + } + int limit = 0; + if (sortRel.fetch != null) { + limit = RexLiteral.intValue(sortRel.fetch); + } + if (limit <= 0) { + // Cannot enable group trim without limit. + return; + } + + PinotLogicalAggregate newAggRel = createPlan(call, aggRel, true, hintOptions, newCollations, limit); + RelNode newProjectRel = projectRel.copy(projectRel.getTraitSet(), List.of(newAggRel)); + call.transformTo(sortRel.copy(sortRel.getTraitSet(), List.of(newProjectRel))); + } } - /** - * Split the AGG into 3 plan fragments, all with the same AGG type (in some cases the final agg name may be different) - * Pinot internal plan fragment optimization can use the info of the input data type to infer whether it should - * generate the "final-stage AGG operator" or "intermediate-stage AGG operator" or "leaf-stage AGG operator" - * - * @param call the {@link RelOptRuleCall} on match. - * @see org.apache.pinot.core.query.aggregation.function.AggregationFunction - */ - @Override - public void onMatch(RelOptRuleCall call) { - Aggregate aggRel = call.rel(0); - boolean hasGroupBy = !aggRel.getGroupSet().isEmpty(); - RelCollation collation = extractWithInGroupCollation(aggRel); - Map hintOptions = - PinotHintStrategyTable.getHintOptions(aggRel.getHints(), PinotHintOptions.AGGREGATE_HINT_OPTIONS); - // Collation is not supported in leaf stage aggregation. - if (collation != null || (hasGroupBy && hintOptions != null && Boolean.parseBoolean( + public static class SortAggregate extends RelOptRule { + public static final SortAggregate INSTANCE = new SortAggregate(PinotRuleUtils.PINOT_REL_FACTORY); + + private SortAggregate(RelBuilderFactory factory) { + // NOTE: Explicitly match for LogicalAggregate because after applying the rule, LogicalAggregate is replaced with + // PinotLogicalAggregate, and the rule won't be applied again. + super(operand(Sort.class, operand(LogicalAggregate.class, any())), factory, null); + } + + @Override + public void onMatch(RelOptRuleCall call) { + // Apply this rule for group-by queries with enable group trim hint. + LogicalAggregate aggRel = call.rel(1); + if (aggRel.getGroupSet().isEmpty()) { + return; + } + Map hintOptions = + PinotHintStrategyTable.getHintOptions(aggRel.getHints(), PinotHintOptions.AGGREGATE_HINT_OPTIONS); + if (hintOptions == null || !Boolean.parseBoolean( + hintOptions.get(PinotHintOptions.AggregateOptions.ENABLE_GROUP_TRIM))) { + return; + } + + Sort sortRel = call.rel(0); + List collations = sortRel.getCollation().getFieldCollations(); + if (collations.isEmpty()) { + // Cannot enable group trim without sort key. + return; + } + int limit = 0; + if (sortRel.fetch != null) { + limit = RexLiteral.intValue(sortRel.fetch); + } + if (limit <= 0) { + // Cannot enable group trim without limit. + return; + } + + PinotLogicalAggregate newAggRel = createPlan(call, aggRel, true, hintOptions, collations, limit); + call.transformTo(sortRel.copy(sortRel.getTraitSet(), List.of(newAggRel))); + } + } + + public static class WithoutSort extends RelOptRule { + public static final WithoutSort INSTANCE = new WithoutSort(PinotRuleUtils.PINOT_REL_FACTORY); + + private WithoutSort(RelBuilderFactory factory) { + // NOTE: Explicitly match for LogicalAggregate because after applying the rule, LogicalAggregate is replaced with + // PinotLogicalAggregate, and the rule won't be applied again. + super(operand(LogicalAggregate.class, any()), factory, null); + } + + @Override + public void onMatch(RelOptRuleCall call) { + Aggregate aggRel = call.rel(0); + Map hintOptions = + PinotHintStrategyTable.getHintOptions(aggRel.getHints(), PinotHintOptions.AGGREGATE_HINT_OPTIONS); + call.transformTo( + createPlan(call, aggRel, !aggRel.getGroupSet().isEmpty(), hintOptions != null ? hintOptions : Map.of(), null, + 0)); + } + } + + private static PinotLogicalAggregate createPlan(RelOptRuleCall call, Aggregate aggRel, boolean hasGroupBy, + Map hintOptions, @Nullable List collations, int limit) { + // WITHIN GROUP collation is not supported in leaf stage aggregation. + RelCollation withinGroupCollation = extractWithinGroupCollation(aggRel); + if (withinGroupCollation != null || (hasGroupBy && Boolean.parseBoolean( hintOptions.get(PinotHintOptions.AggregateOptions.SKIP_LEAF_STAGE_GROUP_BY_AGGREGATION)))) { - call.transformTo(createPlanWithExchangeDirectAggregation(call, collation)); - } else if (hasGroupBy && hintOptions != null && Boolean.parseBoolean( + return createPlanWithExchangeDirectAggregation(call, aggRel, withinGroupCollation, collations, limit); + } else if (hasGroupBy && Boolean.parseBoolean( hintOptions.get(PinotHintOptions.AggregateOptions.IS_PARTITIONED_BY_GROUP_BY_KEYS))) { - call.transformTo(new PinotLogicalAggregate(aggRel, buildAggCalls(aggRel, AggType.DIRECT, false), AggType.DIRECT)); + return new PinotLogicalAggregate(aggRel, aggRel.getInput(), buildAggCalls(aggRel, AggType.DIRECT, false), + AggType.DIRECT, false, collations, limit); } else { - boolean leafReturnFinalResult = hintOptions != null && Boolean.parseBoolean( - hintOptions.get(PinotHintOptions.AggregateOptions.IS_LEAF_RETURN_FINAL_RESULT)); - call.transformTo(createPlanWithLeafExchangeFinalAggregate(call, leafReturnFinalResult)); + boolean leafReturnFinalResult = + Boolean.parseBoolean(hintOptions.get(PinotHintOptions.AggregateOptions.IS_LEAF_RETURN_FINAL_RESULT)); + return createPlanWithLeafExchangeFinalAggregate(aggRel, leafReturnFinalResult, collations, limit); } } // TODO: Currently it only handles one WITHIN GROUP collation across all AggregateCalls. @Nullable - private static RelCollation extractWithInGroupCollation(Aggregate aggRel) { + private static RelCollation extractWithinGroupCollation(Aggregate aggRel) { for (AggregateCall aggCall : aggRel.getAggCallList()) { RelCollation collation = aggCall.getCollation(); if (!collation.getFieldCollations().isEmpty()) { @@ -138,55 +240,54 @@ private static RelCollation extractWithInGroupCollation(Aggregate aggRel) { * Use this group by optimization to skip leaf stage aggregation when aggregating at leaf level is not desired. Many * situation could be wasted effort to do group-by on leaf, eg: when cardinality of group by column is very high. */ - private static PinotLogicalAggregate createPlanWithExchangeDirectAggregation(RelOptRuleCall call, - @Nullable RelCollation collation) { - Aggregate aggRel = call.rel(0); + private static PinotLogicalAggregate createPlanWithExchangeDirectAggregation(RelOptRuleCall call, Aggregate aggRel, + @Nullable RelCollation withinGroupCollation, @Nullable List collations, int limit) { RelNode input = aggRel.getInput(); // Create Project when there's none below the aggregate. if (!(PinotRuleUtils.unboxRel(input) instanceof Project)) { - aggRel = (Aggregate) generateProjectUnderAggregate(call); + aggRel = (Aggregate) generateProjectUnderAggregate(call, aggRel); input = aggRel.getInput(); } ImmutableBitSet groupSet = aggRel.getGroupSet(); RelDistribution distribution = RelDistributions.hash(groupSet.asList()); RelNode exchange; - if (collation != null) { + if (withinGroupCollation != null) { // Insert a LogicalSort node between exchange and aggregate whe collation exists. - exchange = PinotLogicalSortExchange.create(input, distribution, collation, false, true); + exchange = PinotLogicalSortExchange.create(input, distribution, withinGroupCollation, false, true); } else { exchange = PinotLogicalExchange.create(input, distribution); } - return new PinotLogicalAggregate(aggRel, exchange, buildAggCalls(aggRel, AggType.DIRECT, false), AggType.DIRECT); + return new PinotLogicalAggregate(aggRel, exchange, buildAggCalls(aggRel, AggType.DIRECT, false), AggType.DIRECT, + false, collations, limit); } /** * Aggregate node will be split into LEAF + EXCHANGE + FINAL. * TODO: Add optional INTERMEDIATE stage to reduce hotspot. */ - private static PinotLogicalAggregate createPlanWithLeafExchangeFinalAggregate(RelOptRuleCall call, - boolean leafReturnFinalResult) { - Aggregate aggRel = call.rel(0); + private static PinotLogicalAggregate createPlanWithLeafExchangeFinalAggregate(Aggregate aggRel, + boolean leafReturnFinalResult, @Nullable List collations, int limit) { // Create a LEAF aggregate. PinotLogicalAggregate leafAggRel = - new PinotLogicalAggregate(aggRel, buildAggCalls(aggRel, AggType.LEAF, leafReturnFinalResult), AggType.LEAF, - leafReturnFinalResult); + new PinotLogicalAggregate(aggRel, aggRel.getInput(), buildAggCalls(aggRel, AggType.LEAF, leafReturnFinalResult), + AggType.LEAF, leafReturnFinalResult, collations, limit); // Create an EXCHANGE node over the LEAF aggregate. PinotLogicalExchange exchange = PinotLogicalExchange.create(leafAggRel, RelDistributions.hash(ImmutableIntList.range(0, aggRel.getGroupCount()))); // Create a FINAL aggregate over the EXCHANGE. - return convertAggFromIntermediateInput(call, exchange, AggType.FINAL, leafReturnFinalResult); + return convertAggFromIntermediateInput(aggRel, exchange, AggType.FINAL, leafReturnFinalResult, collations, limit); } /** * The following is copied from {@link AggregateExtractProjectRule#onMatch(RelOptRuleCall)} with modification to take * aggregate input as input. */ - private static RelNode generateProjectUnderAggregate(RelOptRuleCall call) { - final Aggregate aggregate = call.rel(0); + private static RelNode generateProjectUnderAggregate(RelOptRuleCall call, Aggregate aggregate) { // --------------- MODIFIED --------------- final RelNode input = aggregate.getInput(); + // final Aggregate aggregate = call.rel(0); // final RelNode input = call.rel(1); // ------------- END MODIFIED ------------- @@ -230,9 +331,8 @@ private static RelNode generateProjectUnderAggregate(RelOptRuleCall call) { return relBuilder.build(); } - private static PinotLogicalAggregate convertAggFromIntermediateInput(RelOptRuleCall call, - PinotLogicalExchange exchange, AggType aggType, boolean leafReturnFinalResult) { - Aggregate aggRel = call.rel(0); + private static PinotLogicalAggregate convertAggFromIntermediateInput(Aggregate aggRel, PinotLogicalExchange exchange, + AggType aggType, boolean leafReturnFinalResult, @Nullable List collations, int limit) { RelNode input = aggRel.getInput(); List projects = findImmediateProjects(input); @@ -269,7 +369,7 @@ private static PinotLogicalAggregate convertAggFromIntermediateInput(RelOptRuleC } return new PinotLogicalAggregate(aggRel, exchange, ImmutableBitSet.range(groupCount), aggCalls, aggType, - leafReturnFinalResult); + leafReturnFinalResult, collations, limit); } private static List buildAggCalls(Aggregate aggRel, AggType aggType, boolean leafReturnFinalResult) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java index fdb75ee78f19..630574b18398 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/rules/PinotQueryRuleSets.java @@ -137,7 +137,9 @@ private PinotQueryRuleSets() { PinotSingleValueAggregateRemoveRule.INSTANCE, PinotJoinExchangeNodeInsertRule.INSTANCE, - PinotAggregateExchangeNodeInsertRule.INSTANCE, + PinotAggregateExchangeNodeInsertRule.SortProjectAggregate.INSTANCE, + PinotAggregateExchangeNodeInsertRule.SortAggregate.INSTANCE, + PinotAggregateExchangeNodeInsertRule.WithoutSort.INSTANCE, PinotWindowExchangeNodeInsertRule.INSTANCE, PinotSetOpExchangeNodeInsertRule.INSTANCE, diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/parser/CalciteRexExpressionParser.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/parser/CalciteRexExpressionParser.java index a20b2479d4f0..fdd19a9aef23 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/parser/CalciteRexExpressionParser.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/parser/CalciteRexExpressionParser.java @@ -29,7 +29,6 @@ import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.common.utils.request.RequestUtils; import org.apache.pinot.query.planner.logical.RexExpression; -import org.apache.pinot.query.planner.plannode.SortNode; import org.apache.pinot.spi.utils.BooleanUtils; import org.apache.pinot.spi.utils.ByteArray; import org.apache.pinot.sql.parsers.ParserUtils; @@ -96,8 +95,7 @@ public static List convertAggregateList(List groupByList return expressions; } - public static List convertOrderByList(SortNode node, PinotQuery pinotQuery) { - List collations = node.getCollations(); + public static List convertOrderByList(List collations, PinotQuery pinotQuery) { List orderByExpressions = new ArrayList<>(collations.size()); for (RelFieldCollation collation : collations) { orderByExpressions.add(convertOrderBy(collation, pinotQuery)); diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java index 611d4417259b..6ae02da45fc9 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/explain/PlanNodeMerger.java @@ -147,6 +147,12 @@ public PlanNode visitAggregate(AggregateNode node, PlanNode context) { if (node.isLeafReturnFinalResult() != otherNode.isLeafReturnFinalResult()) { return null; } + if (!node.getCollations().equals(otherNode.getCollations())) { + return null; + } + if (node.getLimit() != otherNode.getLimit()) { + return null; + } List children = mergeChildren(node, context); if (children == null) { return null; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/EquivalentStagesFinder.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/EquivalentStagesFinder.java index 55813264ffb0..61cf5d5be626 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/EquivalentStagesFinder.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/EquivalentStagesFinder.java @@ -195,7 +195,9 @@ public Boolean visitAggregate(AggregateNode node1, PlanNode node2) { && Objects.equals(node1.getFilterArgs(), that.getFilterArgs()) && Objects.equals(node1.getGroupKeys(), that.getGroupKeys()) && node1.getAggType() == that.getAggType() - && node1.isLeafReturnFinalResult() == that.isLeafReturnFinalResult(); + && node1.isLeafReturnFinalResult() == that.isLeafReturnFinalResult() + && Objects.equals(node1.getCollations(), that.getCollations()) + && node1.getLimit() == that.getLimit(); } @Override diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java index 38170116126a..3f5ab2261e0c 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/logical/RelToPlanNodeConverter.java @@ -264,7 +264,7 @@ private AggregateNode convertLogicalAggregate(PinotLogicalAggregate node) { } return new AggregateNode(DEFAULT_STAGE_ID, toDataSchema(node.getRowType()), NodeHint.fromRelHints(node.getHints()), convertInputs(node.getInputs()), functionCalls, filterArgs, node.getGroupSet().asList(), node.getAggType(), - node.isLeafReturnFinalResult()); + node.isLeafReturnFinalResult(), node.getCollations(), node.getLimit()); } private ProjectNode convertLogicalProject(LogicalProject node) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java index be4a6d9fb87d..323c58cce225 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/plannode/AggregateNode.java @@ -20,6 +20,8 @@ import java.util.List; import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.calcite.rel.RelFieldCollation; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.query.planner.logical.RexExpression; @@ -31,15 +33,21 @@ public class AggregateNode extends BasePlanNode { private final AggType _aggType; private final boolean _leafReturnFinalResult; + // The following fields are for group trimming purpose, and are extracted from the Sort on top of this Aggregate. + private final List _collations; + private final int _limit; + public AggregateNode(int stageId, DataSchema dataSchema, NodeHint nodeHint, List inputs, List aggCalls, List filterArgs, List groupKeys, AggType aggType, - boolean leafReturnFinalResult) { + boolean leafReturnFinalResult, @Nullable List collations, int limit) { super(stageId, dataSchema, nodeHint, inputs); _aggCalls = aggCalls; _filterArgs = filterArgs; _groupKeys = groupKeys; _aggType = aggType; _leafReturnFinalResult = leafReturnFinalResult; + _collations = collations != null ? collations : List.of(); + _limit = limit; } public List getAggCalls() { @@ -62,6 +70,14 @@ public boolean isLeafReturnFinalResult() { return _leafReturnFinalResult; } + public List getCollations() { + return _collations; + } + + public int getLimit() { + return _limit; + } + @Override public String explain() { return "AGGREGATE_" + _aggType; @@ -75,7 +91,7 @@ public T visit(PlanNodeVisitor visitor, C context) { @Override public PlanNode withInputs(List inputs) { return new AggregateNode(_stageId, _dataSchema, _nodeHint, inputs, _aggCalls, _filterArgs, _groupKeys, _aggType, - _leafReturnFinalResult); + _leafReturnFinalResult, _collations, _limit); } @Override @@ -90,14 +106,15 @@ public boolean equals(Object o) { return false; } AggregateNode that = (AggregateNode) o; - return Objects.equals(_aggCalls, that._aggCalls) && Objects.equals(_filterArgs, that._filterArgs) && Objects.equals( - _groupKeys, that._groupKeys) && _aggType == that._aggType - && _leafReturnFinalResult == that._leafReturnFinalResult; + return _leafReturnFinalResult == that._leafReturnFinalResult && _limit == that._limit && Objects.equals(_aggCalls, + that._aggCalls) && Objects.equals(_filterArgs, that._filterArgs) && Objects.equals(_groupKeys, that._groupKeys) + && _aggType == that._aggType && Objects.equals(_collations, that._collations); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), _aggCalls, _filterArgs, _groupKeys, _aggType, _leafReturnFinalResult); + return Objects.hash(super.hashCode(), _aggCalls, _filterArgs, _groupKeys, _aggType, _leafReturnFinalResult, + _collations, _limit); } /** diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeDeserializer.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeDeserializer.java index abd474ebce3e..0f6851418925 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeDeserializer.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeDeserializer.java @@ -87,7 +87,8 @@ private static AggregateNode deserializeAggregateNode(Plan.PlanNode protoNode) { return new AggregateNode(protoNode.getStageId(), extractDataSchema(protoNode), extractNodeHint(protoNode), extractInputs(protoNode), convertFunctionCalls(protoAggregateNode.getAggCallsList()), protoAggregateNode.getFilterArgsList(), protoAggregateNode.getGroupKeysList(), - convertAggType(protoAggregateNode.getAggType()), protoAggregateNode.getLeafReturnFinalResult()); + convertAggType(protoAggregateNode.getAggType()), protoAggregateNode.getLeafReturnFinalResult(), + convertCollations(protoAggregateNode.getCollationsList()), protoAggregateNode.getLimit()); } private static FilterNode deserializeFilterNode(Plan.PlanNode protoNode) { diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeSerializer.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeSerializer.java index 65ccb13b2cae..e7862173e749 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeSerializer.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/planner/serde/PlanNodeSerializer.java @@ -98,6 +98,8 @@ public Void visitAggregate(AggregateNode node, Plan.PlanNode.Builder builder) { .addAllGroupKeys(node.getGroupKeys()) .setAggType(convertAggType(node.getAggType())) .setLeafReturnFinalResult(node.isLeafReturnFinalResult()) + .addAllCollations(convertCollations(node.getCollations())) + .setLimit(node.getLimit()) .build(); builder.setAggregateNode(aggregateNode); return null; diff --git a/pinot-query-planner/src/test/resources/queries/GroupByPlans.json b/pinot-query-planner/src/test/resources/queries/GroupByPlans.json index 63a69f5e8ecb..8baa87a06f5d 100644 --- a/pinot-query-planner/src/test/resources/queries/GroupByPlans.json +++ b/pinot-query-planner/src/test/resources/queries/GroupByPlans.json @@ -249,6 +249,39 @@ "\n LogicalTableScan(table=[[default, a]])", "\n" ] + }, + { + "description": "SQL hint based group by optimization with partitioned aggregated values and group trim enabled", + "sql": "EXPLAIN PLAN FOR SELECT /*+ aggOptions(is_leaf_return_final_result='true', is_enable_group_trim='true') */ col1, COUNT(DISTINCT col2) AS cnt FROM a WHERE a.col3 >= 0 GROUP BY col1 ORDER BY cnt DESC LIMIT 10", + "output": [ + "Execution Plan", + "\nLogicalSort(sort0=[$1], dir0=[DESC], offset=[0], fetch=[10])", + "\n PinotLogicalSortExchange(distribution=[hash], collation=[[1 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", + "\n LogicalSort(sort0=[$1], dir0=[DESC], fetch=[10])", + "\n PinotLogicalAggregate(group=[{0}], agg#0=[DISTINCTCOUNT($1)], aggType=[FINAL], leafReturnFinalResult=[true], collations=[[1 DESC]], limit=[10])", + "\n PinotLogicalExchange(distribution=[hash[0]])", + "\n PinotLogicalAggregate(group=[{0}], agg#0=[DISTINCTCOUNT($1)], aggType=[LEAF], leafReturnFinalResult=[true], collations=[[1 DESC]], limit=[10])", + "\n LogicalFilter(condition=[>=($2, 0)])", + "\n LogicalTableScan(table=[[default, a]])", + "\n" + ] + }, + { + "description": "SQL hint based group by optimization with group trim enabled without returning group key", + "sql": "EXPLAIN PLAN FOR SELECT /*+ aggOptions(is_enable_group_trim='true') */ COUNT(DISTINCT col2) AS cnt FROM a WHERE a.col3 >= 0 GROUP BY col1 ORDER BY cnt DESC LIMIT 10", + "output": [ + "Execution Plan", + "\nLogicalSort(sort0=[$0], dir0=[DESC], offset=[0], fetch=[10])", + "\n PinotLogicalSortExchange(distribution=[hash], collation=[[0 DESC]], isSortOnSender=[false], isSortOnReceiver=[true])", + "\n LogicalSort(sort0=[$0], dir0=[DESC], fetch=[10])", + "\n LogicalProject(cnt=[$1])", + "\n PinotLogicalAggregate(group=[{0}], agg#0=[DISTINCTCOUNT($1)], aggType=[FINAL], collations=[[1 DESC]], limit=[10])", + "\n PinotLogicalExchange(distribution=[hash[0]])", + "\n PinotLogicalAggregate(group=[{0}], agg#0=[DISTINCTCOUNT($1)], aggType=[LEAF], collations=[[1 DESC]], limit=[10])", + "\n LogicalFilter(condition=[>=($2, 0)])", + "\n LogicalTableScan(table=[[default, a]])", + "\n" + ] } ] } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestVisitor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestVisitor.java index bd58b7f64f04..254aa2430c93 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestVisitor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestVisitor.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.calcite.rel.RelFieldCollation; import org.apache.pinot.calcite.rel.logical.PinotRelExchangeType; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.request.DataSource; @@ -71,22 +72,27 @@ static void walkPlanNode(PlanNode node, ServerPlanRequestContext context) { public Void visitAggregate(AggregateNode node, ServerPlanRequestContext context) { if (visit(node.getInputs().get(0), context)) { PinotQuery pinotQuery = context.getPinotQuery(); - if (pinotQuery.getGroupByList() == null) { - List groupByList = CalciteRexExpressionParser.convertInputRefs(node.getGroupKeys(), pinotQuery); + List groupByList = CalciteRexExpressionParser.convertInputRefs(node.getGroupKeys(), pinotQuery); + if (!groupByList.isEmpty()) { pinotQuery.setGroupByList(groupByList); - pinotQuery.setSelectList( - CalciteRexExpressionParser.convertAggregateList(groupByList, node.getAggCalls(), node.getFilterArgs(), - pinotQuery)); - if (node.getAggType() == AggregateNode.AggType.DIRECT) { - pinotQuery.putToQueryOptions(CommonConstants.Broker.Request.QueryOptionKey.SERVER_RETURN_FINAL_RESULT, - "true"); - } else if (node.isLeafReturnFinalResult()) { - pinotQuery.putToQueryOptions( - CommonConstants.Broker.Request.QueryOptionKey.SERVER_RETURN_FINAL_RESULT_KEY_UNPARTITIONED, "true"); - } - // there cannot be any more modification of PinotQuery post agg, thus this is the last one possible. - context.setLeafStageBoundaryNode(node); } + pinotQuery.setSelectList( + CalciteRexExpressionParser.convertAggregateList(groupByList, node.getAggCalls(), node.getFilterArgs(), + pinotQuery)); + if (node.getAggType() == AggregateNode.AggType.DIRECT) { + pinotQuery.putToQueryOptions(CommonConstants.Broker.Request.QueryOptionKey.SERVER_RETURN_FINAL_RESULT, "true"); + } else if (node.isLeafReturnFinalResult()) { + pinotQuery.putToQueryOptions( + CommonConstants.Broker.Request.QueryOptionKey.SERVER_RETURN_FINAL_RESULT_KEY_UNPARTITIONED, "true"); + } + List collations = node.getCollations(); + int limit = node.getLimit(); + if (!collations.isEmpty() && limit > 0) { + pinotQuery.setOrderByList(CalciteRexExpressionParser.convertOrderByList(collations, pinotQuery)); + pinotQuery.setLimit(limit); + } + // There cannot be any more modification of PinotQuery post agg, thus this is the last one possible. + context.setLeafStageBoundaryNode(node); } return null; } @@ -193,8 +199,9 @@ public Void visitSort(SortNode node, ServerPlanRequestContext context) { if (visit(node.getInputs().get(0), context)) { PinotQuery pinotQuery = context.getPinotQuery(); if (pinotQuery.getOrderByList() == null) { - if (!node.getCollations().isEmpty()) { - pinotQuery.setOrderByList(CalciteRexExpressionParser.convertOrderByList(node, pinotQuery)); + List collations = node.getCollations(); + if (!collations.isEmpty()) { + pinotQuery.setOrderByList(CalciteRexExpressionParser.convertOrderByList(collations, pinotQuery)); } if (node.getFetch() >= 0) { pinotQuery.setLimit(node.getFetch()); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java index f7f56e0ccb6e..b2e73f226a3a 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java @@ -273,7 +273,7 @@ private AggregateOperator getOperator(DataSchema resultSchema, List filterArgs, List groupKeys, PlanNode.NodeHint nodeHint) { return new AggregateOperator(OperatorTestUtil.getTracingContext(), _input, new AggregateNode(-1, resultSchema, nodeHint, List.of(), aggCalls, filterArgs, groupKeys, AggType.DIRECT, - false)); + false, null, 0)); } private AggregateOperator getOperator(DataSchema resultSchema, List aggCalls, diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MultiStageAccountingTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MultiStageAccountingTest.java index fc7ebba0b4cb..05ccf5762191 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MultiStageAccountingTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/MultiStageAccountingTest.java @@ -152,7 +152,7 @@ private static MultiStageOperator getAggregateOperator() { new DataSchema(new String[]{"group", "sum"}, new DataSchema.ColumnDataType[]{INT, DOUBLE}); return new AggregateOperator(OperatorTestUtil.getTracingContext(), input, new AggregateNode(-1, resultSchema, PlanNode.NodeHint.EMPTY, List.of(), aggCalls, filterArgs, groupKeys, - AggregateNode.AggType.DIRECT, false)); + AggregateNode.AggType.DIRECT, false, null, 0)); } private static MultiStageOperator getHashJoinOperator() { diff --git a/pinot-query-runtime/src/test/resources/queries/QueryHints.json b/pinot-query-runtime/src/test/resources/queries/QueryHints.json index e7c2ca375700..f364274db6ea 100644 --- a/pinot-query-runtime/src/test/resources/queries/QueryHints.json +++ b/pinot-query-runtime/src/test/resources/queries/QueryHints.json @@ -321,6 +321,10 @@ "description": "aggregate with skip intermediate stage hint (via hint option is_partitioned_by_group_by_keys)", "sql": "SELECT /*+ aggOptions(is_partitioned_by_group_by_keys='true') */ {tbl1}.num, COUNT(*), SUM({tbl1}.val), SUM({tbl1}.num), COUNT(DISTINCT {tbl1}.val) FROM {tbl1} WHERE {tbl1}.val >= 0 AND {tbl1}.name != 'a' GROUP BY {tbl1}.num" }, + { + "description": "aggregate with skip intermediate stage hint and group trim enabled", + "sql": "SELECT /*+ aggOptions(is_partitioned_by_group_by_keys='true', is_enable_group_trim='true') */ {tbl1}.num, COUNT(*), SUM({tbl1}.val), SUM({tbl1}.num), COUNT(DISTINCT {tbl1}.val) FROM {tbl1} WHERE {tbl1}.val >= 0 AND {tbl1}.name != 'a' GROUP BY {tbl1}.num ORDER BY COUNT(*) DESC, {tbl1}.num LIMIT 1" + }, { "description": "join with pre-partitioned left and right tables", "sql": "SELECT {tbl1}.num, {tbl1}.val, {tbl2}.data FROM {tbl1} /*+ tableOptions(partition_function='hashcode', partition_key='num', partition_size='4') */ JOIN {tbl2} /*+ tableOptions(partition_function='hashcode', partition_key='num', partition_size='4') */ ON {tbl1}.num = {tbl2}.num WHERE {tbl2}.data > 0" From b603ad5ef34c6316201a9396424cdb1924a9dc0a Mon Sep 17 00:00:00 2001 From: Bolek Ziobrowski <26925920+bziobrowski@users.noreply.github.com> Date: Mon, 30 Dec 2024 11:02:45 +0100 Subject: [PATCH 2/6] Add group_trim_size and error_on_num_groups_limit hints/options. --- .../utils/config/QueryOptionsUtils.java | 4 + .../helix/ControllerRequestClient.java | 38 + .../controller/helix/ControllerTest.java | 5 + .../operator/query/AggregationOperator.java | 2 +- .../core/operator/query/GroupByOperator.java | 2 +- .../pinot/core/plan/CombinePlanNode.java | 3 +- .../DictionaryBasedGroupKeyGenerator.java | 9 +- .../query/reduce/BrokerReduceService.java | 77 +- .../query/reduce/GroupByDataTableReducer.java | 10 +- .../query/request/context/QueryContext.java | 3 +- .../utils/QueryContextConverterUtils.java | 22 +- .../apache/pinot/core/util/GroupByUtils.java | 3 +- .../function/AvgAggregationFunctionTest.java | 75 ++ .../tests/GroupByOptionsIntegrationTest.java | 681 ++++++++++++++++++ .../calcite/rel/hint/PinotHintOptions.java | 11 + .../rel/logical/PinotLogicalSortExchange.java | 2 +- .../runtime/operator/AggregateOperator.java | 93 ++- .../MultistageAggregationExecutor.java | 3 +- .../operator/MultistageGroupByExecutor.java | 90 ++- .../plan/server/ServerPlanRequestUtils.java | 19 +- .../service/dispatch/QueryDispatcher.java | 42 +- .../pinot/spi/utils/CommonConstants.java | 3 + .../builder/ControllerRequestURLBuilder.java | 4 + .../spi/utils/builder/TableConfigBuilder.java | 9 + 24 files changed, 1110 insertions(+), 100 deletions(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java index 32f97a0c1448..e29da5cc6d08 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java @@ -268,6 +268,10 @@ public static Integer getMultiStageLeafLimit(Map queryOptions) { return checkedParseIntNonNegative(QueryOptionKey.MULTI_STAGE_LEAF_LIMIT, maxLeafLimitStr); } + public static boolean getErrorOnNumGroupsLimit(Map queryOptions) { + return Boolean.parseBoolean(queryOptions.get(QueryOptionKey.ERROR_ON_NUM_GROUPS_LIMIT)); + } + @Nullable public static Integer getNumGroupsLimit(Map queryOptions) { String maxNumGroupLimit = queryOptions.get(QueryOptionKey.NUM_GROUPS_LIMIT); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java index 5f8f7d3190fc..311a1caadad2 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java @@ -25,6 +25,8 @@ import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import javax.annotation.Nullable; @@ -244,6 +246,42 @@ public List listSegments(String tableName, @Nullable String tableType, b } } + public Map> getServersToSegmentsMap(String tableName, TableType tableType) + throws IOException { + String url = _controllerRequestURLBuilder.forServersToSegmentsMap(tableName, tableType.toString()); + try { + SimpleHttpResponse resp = + HttpClient.wrapAndThrowHttpException(_httpClient.sendGetRequest(new URI(url), _headers)); + JsonNode jsonNode = JsonUtils.stringToJsonNode(resp.getResponse()); + if (jsonNode == null || jsonNode.get(0) == null) { + return Collections.emptyMap(); + } + + JsonNode serversMap = jsonNode.get(0).get("serverToSegmentsMap"); + if (serversMap == null) { + return Collections.emptyMap(); + } + + HashMap> result = new HashMap<>(); + Iterator> fields = serversMap.fields(); + while (fields.hasNext()) { + Map.Entry field = fields.next(); + List segments = new ArrayList<>(); + + ArrayNode value = (ArrayNode) field.getValue(); + for (int i = 0, len = value.size(); i < len; i++) { + segments.add(value.get(i).toString()); + } + + result.put(field.getKey(), segments); + } + + return result; + } catch (HttpErrorStatusException | URISyntaxException e) { + throw new IOException(e); + } + } + public void deleteSegment(String tableName, String segmentName) throws IOException { try { diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java index c0a3230e8596..1208c9b9cfcc 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java @@ -728,6 +728,11 @@ public long getTableSize(String tableName) return getControllerRequestClient().getTableSize(tableName); } + public Map> getTableServersToSegmentsMap(String tableName, TableType tableType) + throws IOException { + return getControllerRequestClient().getServersToSegmentsMap(tableName, tableType); + } + public String reloadOfflineTable(String tableName) throws IOException { return reloadOfflineTable(tableName, false); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/query/AggregationOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/query/AggregationOperator.java index c1a2aa157a40..31ef246eb328 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/query/AggregationOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/query/AggregationOperator.java @@ -38,7 +38,7 @@ /** - * The AggregationOperator class provides the operator for aggregation only query on a single segment. + * The AggregationOperator class implements keyless aggregation query on a single segment in V1/SSQE. */ @SuppressWarnings("rawtypes") public class AggregationOperator extends BaseOperator { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/query/GroupByOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/query/GroupByOperator.java index 9fae5459be21..6e27c6b36564 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/query/GroupByOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/query/GroupByOperator.java @@ -46,7 +46,7 @@ /** - * The GroupByOperator class provides the operator for group-by query on a single segment. + * The GroupByOperator class implements keyed aggregation on a single segment in V1/SSQE. */ @SuppressWarnings("rawtypes") public class GroupByOperator extends BaseOperator { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java index 26a92082259f..5ac0c79a1a71 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/CombinePlanNode.java @@ -48,7 +48,8 @@ /** - * The CombinePlanNode class provides the execution plan for combining results from multiple segments. + * The CombinePlanNode class provides the execution plan for combining results from multiple segments in + * V1/SSQE. */ @SuppressWarnings({"rawtypes", "unchecked"}) public class CombinePlanNode implements PlanNode { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGenerator.java b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGenerator.java index 257e95c00401..8c55582cb8ba 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGenerator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/groupby/DictionaryBasedGroupKeyGenerator.java @@ -53,7 +53,7 @@ * integer raw keys and map them onto contiguous group ids. (INT_MAP_BASED) * *
  • - * If the maximum number of possible group keys cannot fit into than integer, but still fit into long, generate long + * If the maximum number of possible group keys cannot fit into integer, but still fit into long, generate long * raw keys and map them onto contiguous group ids. (LONG_MAP_BASED) *
  • *
  • @@ -105,8 +105,6 @@ public class DictionaryBasedGroupKeyGenerator implements GroupKeyGenerator { public DictionaryBasedGroupKeyGenerator(BaseProjectOperator projectOperator, ExpressionContext[] groupByExpressions, int numGroupsLimit, int arrayBasedThreshold, @Nullable Map groupByExpressionSizesFromPredicates) { - assert numGroupsLimit >= arrayBasedThreshold; - _groupByExpressions = groupByExpressions; _numGroupByExpressions = groupByExpressions.length; @@ -173,7 +171,9 @@ public DictionaryBasedGroupKeyGenerator(BaseProjectOperator projectOperator, _rawKeyHolder = new LongMapBasedHolder(groupIdMap); } else { _globalGroupIdUpperBound = Math.min((int) cardinalityProduct, numGroupsLimit); - if (cardinalityProduct > arrayBasedThreshold) { + // arrayBaseHolder fails with ArrayIndexOutOfBoundsException if numGroupsLimit < cardinalityProduct + // because array doesn't fit all (potentially unsorted) values + if (cardinalityProduct > arrayBasedThreshold || numGroupsLimit < cardinalityProduct) { // IntMapBasedHolder IntGroupIdMap groupIdMap = THREAD_LOCAL_INT_MAP.get(); groupIdMap.clearAndTrim(); @@ -281,6 +281,7 @@ private interface RawKeyHolder { int getNumKeys(); } + // This holder works only if it can fit all results, otherwise it fails on AIOOBE or produces too many group keys private class ArrayBasedHolder implements RawKeyHolder { private final boolean[] _flags = new boolean[_globalGroupIdUpperBound]; private int _numKeys = 0; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index d10e0811ede9..ccfe789fe52d 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -58,8 +58,12 @@ public BrokerReduceService(PinotConfiguration config) { super(config); } - public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, BrokerRequest serverBrokerRequest, - Map dataTableMap, long reduceTimeOutMs, BrokerMetrics brokerMetrics) { + public BrokerResponseNative reduceOnDataTable( + BrokerRequest brokerRequest, + BrokerRequest serverBrokerRequest, + Map dataTableMap, + long reduceTimeOutMs, + BrokerMetrics brokerMetrics) { if (dataTableMap.isEmpty()) { // Empty response. return BrokerResponseNative.empty(); @@ -70,12 +74,12 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke queryOptions != null && Boolean.parseBoolean(queryOptions.get(CommonConstants.Broker.Request.TRACE)); ExecutionStatsAggregator aggregator = new ExecutionStatsAggregator(enableTrace); - BrokerResponseNative brokerResponseNative = new BrokerResponseNative(); + BrokerResponseNative response = new BrokerResponseNative(); // Cache a data schema from data tables (try to cache one with data rows associated with it). - DataSchema dataSchemaFromEmptyDataTable = null; - DataSchema dataSchemaFromNonEmptyDataTable = null; - List serversWithConflictingDataSchema = new ArrayList<>(); + DataSchema schemaOfEmptyTable = null; + DataSchema schemaOfNonEmptyTable = null; + List serversWithConflictingSchema = new ArrayList<>(); // Process server response metadata. Iterator> iterator = dataTableMap.entrySet().iterator(); @@ -93,19 +97,19 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke } else { // Try to cache a data table with data rows inside, or cache one with data schema inside. if (dataTable.getNumberOfRows() == 0) { - if (dataSchemaFromEmptyDataTable == null) { - dataSchemaFromEmptyDataTable = dataSchema; + if (schemaOfEmptyTable == null) { + schemaOfEmptyTable = dataSchema; } iterator.remove(); } else { - if (dataSchemaFromNonEmptyDataTable == null) { - dataSchemaFromNonEmptyDataTable = dataSchema; + if (schemaOfNonEmptyTable == null) { + schemaOfNonEmptyTable = dataSchema; } else { // Remove data tables with conflicting data schema. // NOTE: Only compare the column data types, since the column names (string representation of expression) // can change across different versions. - if (!Arrays.equals(dataSchema.getColumnDataTypes(), dataSchemaFromNonEmptyDataTable.getColumnDataTypes())) { - serversWithConflictingDataSchema.add(entry.getKey()); + if (!Arrays.equals(dataSchema.getColumnDataTypes(), schemaOfNonEmptyTable.getColumnDataTypes())) { + serversWithConflictingSchema.add(entry.getKey()); iterator.remove(); } } @@ -117,52 +121,51 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke String rawTableName = TableNameBuilder.extractRawTableName(tableName); // Set execution statistics and Update broker metrics. - aggregator.setStats(rawTableName, brokerResponseNative, brokerMetrics); + aggregator.setStats(rawTableName, response, brokerMetrics); // Report the servers with conflicting data schema. - if (!serversWithConflictingDataSchema.isEmpty()) { + if (!serversWithConflictingSchema.isEmpty()) { String errorMessage = QueryException.MERGE_RESPONSE_ERROR.getMessage() + ": responses for table: " + tableName - + " from servers: " + serversWithConflictingDataSchema + " got dropped due to data schema inconsistency."; + + " from servers: " + serversWithConflictingSchema + " got dropped due to data schema inconsistency."; LOGGER.warn(errorMessage); brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.RESPONSE_MERGE_EXCEPTIONS, 1); - brokerResponseNative.addException( - new QueryProcessingException(QueryException.MERGE_RESPONSE_ERROR_CODE, errorMessage)); + response.addException(new QueryProcessingException(QueryException.MERGE_RESPONSE_ERROR_CODE, errorMessage)); } // NOTE: When there is no cached data schema, that means all servers encountered exception. In such case, return the // response with metadata only. DataSchema cachedDataSchema = - dataSchemaFromNonEmptyDataTable != null ? dataSchemaFromNonEmptyDataTable : dataSchemaFromEmptyDataTable; + schemaOfNonEmptyTable != null ? schemaOfNonEmptyTable : schemaOfEmptyTable; if (cachedDataSchema == null) { - return brokerResponseNative; + return response; } - QueryContext serverQueryContext = QueryContextConverterUtils.getQueryContext(serverBrokerRequest.getPinotQuery()); - DataTableReducer dataTableReducer = ResultReducerFactory.getResultReducer(serverQueryContext); + Integer minGroupTrimSizeOpt = null; + Integer groupTrimThresholdOpt = null; + Integer minInitialIndexedTableCapOpt = null; - Integer minGroupTrimSizeQueryOption = null; - Integer groupTrimThresholdQueryOption = null; - Integer minInitialIndexedTableCapacityQueryOption = null; if (queryOptions != null) { - minGroupTrimSizeQueryOption = QueryOptionsUtils.getMinBrokerGroupTrimSize(queryOptions); - groupTrimThresholdQueryOption = QueryOptionsUtils.getGroupTrimThreshold(queryOptions); - minInitialIndexedTableCapacityQueryOption = QueryOptionsUtils.getMinInitialIndexedTableCapacity(queryOptions); + minGroupTrimSizeOpt = QueryOptionsUtils.getMinBrokerGroupTrimSize(queryOptions); + groupTrimThresholdOpt = QueryOptionsUtils.getGroupTrimThreshold(queryOptions); + minInitialIndexedTableCapOpt = QueryOptionsUtils.getMinInitialIndexedTableCapacity(queryOptions); } - int minGroupTrimSize = minGroupTrimSizeQueryOption != null ? minGroupTrimSizeQueryOption : _minGroupTrimSize; - int groupTrimThreshold = - groupTrimThresholdQueryOption != null ? groupTrimThresholdQueryOption : _groupByTrimThreshold; + + int minGroupTrimSize = minGroupTrimSizeOpt != null ? minGroupTrimSizeOpt : _minGroupTrimSize; + int groupTrimThreshold = groupTrimThresholdOpt != null ? groupTrimThresholdOpt : _groupByTrimThreshold; int minInitialIndexedTableCapacity = - minInitialIndexedTableCapacityQueryOption != null ? minInitialIndexedTableCapacityQueryOption - : _minInitialIndexedTableCapacity; + minInitialIndexedTableCapOpt != null ? minInitialIndexedTableCapOpt : _minInitialIndexedTableCapacity; + QueryContext serverQueryContext = QueryContextConverterUtils.getQueryContext(serverBrokerRequest.getPinotQuery()); try { - dataTableReducer.reduceAndSetResults(rawTableName, cachedDataSchema, dataTableMap, brokerResponseNative, + DataTableReducer reducer = ResultReducerFactory.getResultReducer(serverQueryContext); + reducer.reduceAndSetResults(rawTableName, cachedDataSchema, dataTableMap, response, new DataTableReducerContext(_reduceExecutorService, _maxReduceThreadsPerQuery, reduceTimeOutMs, groupTrimThreshold, minGroupTrimSize, minInitialIndexedTableCapacity), brokerMetrics); } catch (EarlyTerminationException e) { - brokerResponseNative.addException( + response.addException( new QueryProcessingException(QueryException.QUERY_CANCELLATION_ERROR_CODE, e.toString())); } + QueryContext queryContext; if (brokerRequest == serverBrokerRequest) { queryContext = serverQueryContext; @@ -173,13 +176,13 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke throw new BadQueryRequestException("Nested query is not supported without gapfill"); } BaseGapfillProcessor gapfillProcessor = GapfillProcessorFactory.getGapfillProcessor(queryContext, gapfillType); - gapfillProcessor.process(brokerResponseNative); + gapfillProcessor.process(response); } if (!serverQueryContext.isExplain()) { - updateAlias(queryContext, brokerResponseNative); + updateAlias(queryContext, response); } - return brokerResponseNative; + return response; } public void shutDown() { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java index d8ff92f90842..d06778538a03 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java @@ -70,6 +70,7 @@ /** * Helper class to reduce data tables and set group by results into the BrokerResponseNative + * Used for key-less aggregations, e.g. select max(id), sum(quantity) from orders . */ @SuppressWarnings("rawtypes") public class GroupByDataTableReducer implements DataTableReducer { @@ -140,9 +141,12 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, * @param brokerMetrics broker metrics (meters) * @throws TimeoutException If unable complete within timeout. */ - private void reduceResult(BrokerResponseNative brokerResponseNative, DataSchema dataSchema, - Collection dataTables, DataTableReducerContext reducerContext, String rawTableName, - BrokerMetrics brokerMetrics) + private void reduceResult(BrokerResponseNative brokerResponseNative, + DataSchema dataSchema, + Collection dataTables, + DataTableReducerContext reducerContext, + String rawTableName, + BrokerMetrics brokerMetrics) throws TimeoutException { // NOTE: This step will modify the data schema and also return final aggregate results. IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java index e1e3c37a8dfd..e5ce066806c0 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/QueryContext.java @@ -207,7 +207,8 @@ public FilterContext getFilter() { } /** - * Returns a list of expressions in the GROUP-BY clause, or {@code null} if there is no GROUP-BY clause. + * Returns a list of expressions in the GROUP-BY clause (aggregation keys), or {@code null} if there is no GROUP-BY + * clause. */ @Nullable public List getGroupByExpressions() { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/QueryContextConverterUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/QueryContextConverterUtils.java index b351ddb0575b..611ffccd5b53 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/QueryContextConverterUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/context/utils/QueryContextConverterUtils.java @@ -166,12 +166,22 @@ public static QueryContext getQueryContext(PinotQuery pinotQuery) { explainMode = ExplainMode.DESCRIPTION; } - return new QueryContext.Builder().setTableName(tableName).setSubquery(subquery) - .setSelectExpressions(selectExpressions).setDistinct(distinct).setAliasList(aliasList).setFilter(filter) - .setGroupByExpressions(groupByExpressions).setOrderByExpressions(orderByExpressions) - .setHavingFilter(havingFilter).setLimit(pinotQuery.getLimit()).setOffset(pinotQuery.getOffset()) - .setQueryOptions(pinotQuery.getQueryOptions()).setExpressionOverrideHints(expressionContextOverrideHints) - .setExplain(explainMode).build(); + return new QueryContext.Builder() + .setTableName(tableName) + .setSubquery(subquery) + .setSelectExpressions(selectExpressions) + .setDistinct(distinct) + .setAliasList(aliasList) + .setFilter(filter) + .setGroupByExpressions(groupByExpressions) + .setOrderByExpressions(orderByExpressions) + .setHavingFilter(havingFilter) + .setLimit(pinotQuery.getLimit()) + .setOffset(pinotQuery.getOffset()) + .setQueryOptions(pinotQuery.getQueryOptions()) + .setExpressionOverrideHints(expressionContextOverrideHints) + .setExplain(explainMode) + .build(); } private static boolean isMultiStage(PinotQuery pinotQuery) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/util/GroupByUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/util/GroupByUtils.java index 313786cecfde..ac25d4a31b8b 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/util/GroupByUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/util/GroupByUtils.java @@ -99,7 +99,8 @@ public static IndexedTable createIndexedTableForCombineOperator(GroupByResultsBl int limit = queryContext.getLimit(); boolean hasOrderBy = queryContext.getOrderByExpressions() != null; boolean hasHaving = queryContext.getHavingFilter() != null; - int minTrimSize = queryContext.getMinServerGroupTrimSize(); + int minTrimSize = + queryContext.getMinServerGroupTrimSize(); // it's minBrokerGroupTrimSize in broker int minInitialIndexedTableCapacity = queryContext.getMinInitialIndexedTableCapacity(); // Disable trim when min trim size is non-positive diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/AvgAggregationFunctionTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/AvgAggregationFunctionTest.java index ddee45428e50..4da450d4cd0c 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/AvgAggregationFunctionTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/aggregation/function/AvgAggregationFunctionTest.java @@ -19,11 +19,16 @@ package org.apache.pinot.core.query.aggregation.function; import org.apache.pinot.queries.FluentQueryTest; +import org.apache.pinot.spi.config.table.FieldConfig; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import static org.apache.pinot.spi.config.table.FieldConfig.CompressionCodec.PASS_THROUGH; + public class AvgAggregationFunctionTest extends AbstractAggregationFunctionTest { @@ -177,4 +182,74 @@ void aggregationGroupByMV(DataTypeScenario scenario) { "tag3 | null" ); } + + @Test(dataProvider = "encodingTypes") + void singleKeyAggregationWithSmallNumGroupsLimitDoesntThrowAIOOBE(FieldConfig.EncodingType encoding) { + FluentQueryTest.withBaseDir(_baseDir) + .givenTable( + new Schema.SchemaBuilder() + .setSchemaName("testTable") + .setEnableColumnBasedNullHandling(true) + .addMetricField("key", FieldSpec.DataType.INT) + .addMetricField("value", FieldSpec.DataType.INT) + .build(), + new TableConfigBuilder(TableType.OFFLINE) + .setTableName("testTable") + .addFieldConfig( + new FieldConfig("key", encoding, (FieldConfig.IndexType) null, PASS_THROUGH, null)) + .build()) + .onFirstInstance(new Object[]{7, 1}, new Object[]{6, 2}, new Object[]{5, 3}, new Object[]{4, 4}) + .andOnSecondInstance(new Object[]{7, 1}, new Object[]{6, 2}, new Object[]{5, 3}, new Object[]{4, 4}) + .whenQuery( + "set numGroupsLimit=3; set maxInitialResultHolderCapacity=1000; " + + "select key, avg(value) " + + "from testTable " + + "group by key " + + "order by key") + .thenResultIs( + "INTEGER | DOUBLE", + "5 | 3", + "6 | 2", + "7 | 1" + ); + } + + @Test(dataProvider = "encodingTypes") + void multiKeyAggregationWithSmallNumGroupsLimitDoesntThrowAIOOBE(FieldConfig.EncodingType encoding) { + FluentQueryTest.withBaseDir(_baseDir) + .givenTable( + new Schema.SchemaBuilder() + .setSchemaName("testTable") + .setEnableColumnBasedNullHandling(true) + .addMetricField("key1", FieldSpec.DataType.INT) + .addMetricField("key2", FieldSpec.DataType.INT) + .addMetricField("value", FieldSpec.DataType.INT) + .build(), + new TableConfigBuilder(TableType.OFFLINE) + .setTableName("testTable") + .addFieldConfig( + new FieldConfig("key1", encoding, (FieldConfig.IndexType) null, PASS_THROUGH, null)) + .addFieldConfig( + new FieldConfig("key2", encoding, (FieldConfig.IndexType) null, PASS_THROUGH, null)) + .build()) + .onFirstInstance(new Object[]{7, 1}, new Object[]{6, 2}, new Object[]{5, 3}, new Object[]{4, 4}) + .andOnSecondInstance(new Object[]{7, 1}, new Object[]{6, 2}, new Object[]{5, 3}, new Object[]{4, 4}) + .whenQuery( + "set numGroupsLimit=3; set maxInitialResultHolderCapacity=1000; " + + "select key1, key2, count(*) " + + "from testTable " + + "group by key1, key2 " + + "order by key1, key2") + .thenResultIs( + "INTEGER | INTEGER | LONG", + "5 | 3 | 2", + "6 | 2 | 2", + "7 | 1 | 2" + ); + } + + @DataProvider(name = "encodingTypes") + FieldConfig.EncodingType[] encodingTypes() { + return FieldConfig.EncodingType.values(); + } } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java new file mode 100644 index 000000000000..715a54a84b9e --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java @@ -0,0 +1,681 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.integration.tests; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.google.common.collect.ImmutableList; +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.apache.pinot.util.TestUtils; +import org.jetbrains.annotations.NotNull; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.apache.pinot.integration.tests.ClusterIntegrationTestUtils.getBrokerQueryApiUrl; + + +public class GroupByOptionsIntegrationTest extends BaseClusterIntegrationTestSet { + + static final int FILES_NO = 4; + static final int RECORDS_NO = 20; + static final String I_COL = "i"; + static final String J_COL = "j"; + static final String RESULT_TABLE = "resultTable"; + static final int SERVERS_NO = 2; + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + startZk(); + startController(); + startServers(SERVERS_NO); + startBroker(); + + Schema schema = new Schema.SchemaBuilder().setSchemaName(DEFAULT_SCHEMA_NAME) + .addSingleValueDimension(I_COL, FieldSpec.DataType.INT) + .addSingleValueDimension(J_COL, FieldSpec.DataType.LONG) + .build(); + addSchema(schema); + TableConfig tableConfig = createOfflineTableConfig(); + addTableConfig(tableConfig); + + List avroFiles = createAvroFile(); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + uploadSegments(DEFAULT_TABLE_NAME, _tarDir); + + // Wait for all documents loaded + TestUtils.waitForCondition(() -> getCurrentCountStarResult(DEFAULT_TABLE_NAME) == FILES_NO * RECORDS_NO, 100L, + 60_000, + "Failed to load documents", true, Duration.ofMillis(60_000 / 10)); + + setUseMultiStageQueryEngine(true); + + Map> map = getTableServersToSegmentsMap(getTableName(), TableType.OFFLINE); + + // make sure segments are split between multiple servers + Assert.assertEquals(map.size(), SERVERS_NO); + } + + protected TableConfig createOfflineTableConfig() { + return new TableConfigBuilder(TableType.OFFLINE) + .setTableName(getTableName()) + .setNumReplicas(getNumReplicas()) + .setBrokerTenant(getBrokerTenant()) + .build(); + } + + private List createAvroFile() + throws IOException { + + // create avro schema + org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("myRecord", null, null, false); + avroSchema.setFields(ImmutableList.of( + new org.apache.avro.Schema.Field(I_COL, + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT), null, null), + new org.apache.avro.Schema.Field(J_COL, + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.LONG), null, null))); + + List files = new ArrayList<>(); + for (int file = 0; file < FILES_NO; file++) { + File avroFile = new File(_tempDir, "data_" + file + ".avro"); + try (DataFileWriter fileWriter = new DataFileWriter<>(new GenericDatumWriter<>(avroSchema))) { + fileWriter.create(avroSchema, avroFile); + + for (int docId = 0; docId < RECORDS_NO; docId++) { + GenericData.Record record = new GenericData.Record(avroSchema); + record.put(I_COL, file); + record.put(J_COL, docId % 10); + fileWriter.append(record); + } + files.add(avroFile); + } + } + return files; + } + + @Test + public void testOrderByKeysIsPushedToFinalAggregationStageWithoutGroupTrimSize() + throws Exception { + // is_enable_group_trim enables V1-style trimming in leaf nodes, + // with numGroupsLimit and minSegmentGroupTrimSize, + // while group_trim_size - in final aggregation node + // NOTE: `set numGroupsLimit=8` global query option applies to both: + // - segment aggregation in leaf stage + // - cross-segment aggregation in intermediate V2 stage + // The latter can easily produce unstable result due to concurrent IndexedTable operation scheduling. + // To stabilize result here, we override it with num_groups_limit hint. + assertResultAndPlan( + // group_trim_size should sort and limit v2 aggregate output if order by and limit is propagated + " set numGroupsLimit=8; set minSegmentGroupTrimSize=7;", + " select /*+ aggOptions(is_enable_group_trim='true',num_groups_limit='100') */ i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i, j desc " + + " limit 1", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "0,\t7,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], offset=[0], fetch=[1])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1 DESC]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], fetch=[1])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0, 1 " + + "DESC]], limit=[1])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n"); + } + + @Test + public void testOrderByKeysIsPushedToFinalAggregationStageWithGroupTrimSize() + throws Exception { + // is_enable_group_trim enables V1-style trimming in leaf nodes, with numGroupsLimit and minSegmentGroupTrimSize, + // while group_trim_size - in final aggregation node . + // Same as above, to stabilize result here, we override global numGroupsLimit option with num_groups_limit hint. + assertResultAndPlan( + // group_trim_size should sort and limit v2 aggregate output if order by and limit is propagated + " set numGroupsLimit=8; set minSegmentGroupTrimSize=7;", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='6',num_groups_limit='20') */ i, j, count" + + "(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i, j desc " + + " limit 1", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "0,\t7,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], offset=[0], fetch=[1])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1 DESC]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC], fetch=[1])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0, 1 " + + "DESC]], limit=[1])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n"); + } + + @Test + public void testOrderByKeysIsPushedToFinalAggregationStage() + throws Exception { + assertResultAndPlan( + // group_trim_size should sort and limit v2 aggregate output if order by and limit is propagated + " ", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='3') */ i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i asc, j asc " + + " limit 3", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "0,\t0,\t2\n" + + "0,\t1,\t2\n" + + "0,\t2,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], offset=[0], fetch=[3])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[3])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0, " + + "1]], limit=[3])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n"); + } + + @Test + public void testHavingOnKeysAndOrderByKeysIsPushedToFinalAggregationStage() + throws Exception { + assertResultAndPlan( + // group_trim_size should sort and limit v2 aggregate output if order by and limit is propagated + " ", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='3') */ i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " having i + j > 10 " + + " order by i asc, j asc " + + " limit 3", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "2,\t9,\t2\n" + + "3,\t8,\t2\n" + + "3,\t9,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], offset=[0], fetch=[3])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[3])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0, " + + "1]], limit=[3])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterExpression(predicate=[plus(i,j) > '10'], operator=[RANGE])\n"); + } + + @Test + public void testGroupByKeysWithOffsetIsPushedToFinalAggregationStage() + throws Exception { + // if offset is set, leaf should return more results to intermediate stage + assertResultAndPlan( + "", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='10') */ i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i asc, j asc " + + " limit 3 " + + " offset 1 ", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "0,\t1,\t2\n" + + "0,\t2,\t2\n" + + "0,\t3,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], offset=[1], fetch=[3])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[4])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0, " + + "1]], limit=[4])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n" + ); + } + + @Test + public void testOrderByByKeysAndValuesIsPushedToFinalAggregationStage() + throws Exception { + // group_trim_size should sort and limit v2 aggregate output if order by and limit is propagated + assertResultAndPlan( + " ", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='3') */ i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i desc, j desc, count(*) desc" + + " limit 3", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "3,\t9,\t2\n" + + "3,\t8,\t2\n" + + "3,\t7,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[DESC], dir1=[DESC], dir2=[DESC], offset=[0]," + + " fetch=[3])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0 DESC, 1 DESC, 2 DESC]], " + + "isSortOnSender=[false], isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[DESC], dir1=[DESC], dir2=[DESC], " + + "fetch=[3])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0 " + + "DESC, 1 DESC, 2 DESC]], limit=[3])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n" + ); + } + + @Test + public void testOrderByKeyValueExpressionIsNotPushedToFinalAggregateStage() + throws Exception { + // Order by both expression based on keys and aggregate values. + // Expression & limit are not available until after aggregation so they can't be pushed down. + // Because of that, group_trim_size is not applied. + // NOTE: order of CombineGroupBy's output is not guaranteed and so is the order of items with equal order by value + // if we change expression to 'order by i + j + count(*) desc' it would be unstable + assertResultAndPlan( + " ", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='3') */ " + + " i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i * j * count(*) desc" + + " limit 3", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "3,\t9,\t2\n" + + "3,\t8,\t2\n" + + "3,\t7,\t2", + "Execution Plan\n" + + "LogicalSort(sort0=[$3], dir0=[DESC], offset=[0], fetch=[3])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[3 DESC]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$3], dir0=[DESC], fetch=[3])\n" + + " LogicalProject(i=[$0], j=[$1], cnt=[$2], EXPR$3=[*(*($0, $1), $2)])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " CombineGroupBy\n" + + " GroupBy(groupKeys=[[i, j]], aggregations=[[count(*)]])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n" + ); + } + + @Test + public void testForGroupByOverJoinOrderByKeyIsPushedToAggregationLeafStage() + throws Exception { + // query uses V2 aggregate operator for both leaf and final stages because of join + assertResultAndPlan( + " ", + " select /*+ aggOptions(is_enable_group_trim='true',group_trim_size='3') */ t1.i, t1.j, count(*) as cnt " + + " from " + getTableName() + " t1 " + + " join " + getTableName() + " t2 on 1=1 " + + " group by t1.i, t1.j " + + " order by t1.i asc, t1.j asc " + + " limit 5", + "\"i\"[\"INT\"],\t\"j\"[\"LONG\"],\t\"cnt\"[\"LONG\"]\n" + + "0,\t0,\t160\n" + + "0,\t1,\t160\n" + + "0,\t2,\t160\n" + + "0,\t3,\t160\n" + + "0,\t4,\t160", + "Execution Plan\n" + + "LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], offset=[0], fetch=[5])\n" + + " PinotLogicalSortExchange(distribution=[hash], collation=[[0, 1]], isSortOnSender=[false], " + + "isSortOnReceiver=[true])\n" + + " LogicalSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC], fetch=[5])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT($2)], aggType=[FINAL], collations=[[0, " + + "1]], limit=[5])\n" + + " PinotLogicalExchange(distribution=[hash[0, 1]])\n" + + " PinotLogicalAggregate(group=[{0, 1}], agg#0=[COUNT()], aggType=[LEAF], collations=[[0, " + + "1]], limit=[5])\n" + + " LogicalJoin(condition=[true], joinType=[inner])\n" + + " PinotLogicalExchange(distribution=[random])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " StreamingCombineSelect\n" + + " SelectStreaming(table=[mytable], totalDocs=[80])\n" + + " Project(columns=[[i, j]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n" + + " PinotLogicalExchange(distribution=[broadcast])\n" + + " LeafStageCombineOperator(table=[mytable])\n" + + " StreamingInstanceResponse\n" + + " StreamingCombineSelect\n" + + " SelectStreaming(table=[mytable], totalDocs=[80])\n" + + " Transform(expressions=[['0']])\n" + + " Project(columns=[[]])\n" + + " DocIdSet(maxDocs=[40000])\n" + + " FilterMatchEntireSegment(numDocs=[80])\n" + ); + } + + public void assertResultAndPlan(String option, String query, String expectedResult, String expectedPlan) + throws Exception { + String sql = option + //disable timeout in debug + + "set timeoutMs=3600000; set brokerReadTimeoutMs=3600000; set brokerConnectTimeoutMs=3600000; " + + query; + + JsonNode result = postV2Query(sql); + JsonNode plan = postV2Query(option + " set explainAskingServers=true; explain plan for " + query); + + Assert.assertEquals(toResultStr(result), new CharSeq(expectedResult)); + Assert.assertEquals(toExplainStr(plan), new CharSeq(expectedPlan)); + } + + @Test + public void testExceptionIsThrownWhenErrorOnNumGroupsLimitHintIsSetAndLimitIsReachedV1() + throws Exception { + String query = " select /*+ aggOptions(num_groups_limit='1',error_on_num_groups_limit='true') */" + + " i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i, j "; + + assertNumGroupsLimitException(query); + } + + @Test + public void testExceptionIsThrownWhenErrorOnNumGroupsLimitHintIsSetAndLimitIsReachedV2() + throws Exception { + String query = " set numGroupsLimit=1;" + + " select /*+ aggOptions(error_on_num_groups_limit='true') */" + + " i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i, j "; + + assertNumGroupsLimitException(query); + } + + @Test + public void testExceptionIsThrownWhenErrorOnNumGroupsLimitOptionIsSetAndLimitIsReachedV1() + throws Exception { + String query = " set errorOnNumGroupsLimit=true; set numGroupsLimit=1;" + + " select i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i, j "; + + assertNumGroupsLimitException(query); + } + + @Test + public void testExceptionIsThrownWhenErrorOnNumGroupsLimitOptionIsSetAndLimitIsReachedV2() + throws Exception { + String query = " set errorOnNumGroupsLimit=true; " + + "select /*+ aggOptions(num_groups_limit='1') */ i, j, count(*) as cnt " + + " from " + getTableName() + + " group by i, j " + + " order by i, j "; + + assertNumGroupsLimitException(query); + } + + private void assertNumGroupsLimitException(String query) + throws Exception { + JsonNode result = postV2Query(query); + + CharSeq errorMessage = toResultStr(result); + + Assert.assertTrue(errorMessage.startsWith("QueryExecutionError:\n" + + "Received error query execution result block: {1000=NUM_GROUPS_LIMIT has been reached at "), + errorMessage.toString()); + } + + // for debug only + protected Properties getPinotConnectionProperties() { + Properties properties = new Properties(); + properties.put("timeoutMs", "3600000"); + properties.put("brokerReadTimeoutMs", "3600000"); + properties.put("brokerConnectTimeoutMs", "3600000"); + properties.putAll(getExtraQueryProperties()); + return properties; + } + + private JsonNode postV2Query(String query) + throws Exception { + return postQuery(query, getBrokerQueryApiUrl(getBrokerBaseApiUrl(), true), null, + getExtraQueryProperties()); + } + + private static @NotNull CharSeq toResultStr(JsonNode mainNode) { + if (mainNode == null) { + return new CharSeq(new StringBuilder("null")); + } + JsonNode node = mainNode.get(RESULT_TABLE); + if (node == null) { + return toErrorString(mainNode.get("exceptions")); + } + return toString(node); + } + + private static @NotNull CharSeq toExplainStr(JsonNode mainNode) { + if (mainNode == null) { + return new CharSeq(new StringBuilder("null")); + } + JsonNode node = mainNode.get(RESULT_TABLE); + if (node == null) { + return toErrorString(mainNode.get("exceptions")); + } + return toExplainString(node); + } + + static class CharSeq implements CharSequence { + private final StringBuilder _sb; + + CharSeq(StringBuilder sb) { + _sb = sb; + } + + CharSeq(String s) { + _sb = new StringBuilder(s); + } + + @Override + public int length() { + return _sb.length(); + } + + @Override + public char charAt(int index) { + return _sb.charAt(index); + } + + public boolean startsWith(CharSequence cs) { + if (cs.length() > _sb.length()) { + return false; + } + + for (int i = 0, len = cs.length(); i < len; i++) { + if (_sb.charAt(i) != cs.charAt(i)) { + return false; + } + } + + return true; + } + + @Override + public @NotNull CharSequence subSequence(int start, int end) { + return new CharSeq(_sb.substring(start, end)); + } + + @NotNull + @Override + public String toString() { + return _sb.toString(); + } + + @Override + public int hashCode() { + int hc = 0; + for (int i = 0, len = _sb.length(); i < len; i++) { + hc = 31 * hc + _sb.charAt(i); + } + return hc; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (!(obj instanceof CharSequence)) { + return false; + } + + CharSequence other = (CharSequence) obj; + if (_sb.length() != other.length()) { + return false; + } + + for (int i = 0, len = _sb.length(); i < len; i++) { + if (_sb.charAt(i) != other.charAt(i)) { + return false; + } + } + + return true; + } + + CharSeq append(CharSequence other) { + _sb.append(other); + return this; + } + } + + public static CharSeq toErrorString(JsonNode node) { + StringBuilder buf = new StringBuilder(); + JsonNode jsonNode = node.get(0); + if (jsonNode != null) { + buf.append(jsonNode.get("message").textValue()); + } + return new CharSeq(buf); + } + + public static CharSeq toString(JsonNode node) { + StringBuilder buf = new StringBuilder(); + ArrayNode columnNames = (ArrayNode) node.get("dataSchema").get("columnNames"); + ArrayNode columnTypes = (ArrayNode) node.get("dataSchema").get("columnDataTypes"); + ArrayNode rows = (ArrayNode) node.get("rows"); + + for (int i = 0; i < columnNames.size(); i++) { + JsonNode name = columnNames.get(i); + JsonNode type = columnTypes.get(i); + + if (i > 0) { + buf.append(",\t"); + } + + buf.append(name).append('[').append(type).append(']'); + } + + for (int i = 0; i < rows.size(); i++) { + ArrayNode row = (ArrayNode) rows.get(i); + + buf.append('\n'); + for (int j = 0; j < row.size(); j++) { + if (j > 0) { + buf.append(",\t"); + } + + buf.append(row.get(j)); + } + } + + return new CharSeq(buf); + } + + public static CharSeq toExplainString(JsonNode node) { + StringBuilder buf = new StringBuilder(); + buf.append(node.get("rows").get(0).get(1).textValue()); + return new CharSeq(buf); + } + + @AfterClass + public void tearDown() + throws Exception { + dropOfflineTable(DEFAULT_TABLE_NAME); + + stopServer(); + stopBroker(); + stopController(); + stopZk(); + + FileUtils.deleteDirectory(_tempDir); + } +} diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java index 82f80da4bee2..e560599a5b10 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java @@ -43,9 +43,20 @@ public static class AggregateOptions { public static final String IS_PARTITIONED_BY_GROUP_BY_KEYS = "is_partitioned_by_group_by_keys"; public static final String IS_LEAF_RETURN_FINAL_RESULT = "is_leaf_return_final_result"; public static final String SKIP_LEAF_STAGE_GROUP_BY_AGGREGATION = "is_skip_leaf_stage_group_by"; + + /** Enables trimming of aggregation intermediate results by pushing down order by and limit to leaf stage. */ public static final String ENABLE_GROUP_TRIM = "is_enable_group_trim"; + /** Throw an exception on reaching num_groups_limit instead of just setting a flag. */ + public static final String ERROR_ON_NUM_GROUPS_LIMIT = "error_on_num_groups_limit"; + + /** Max number of keys produced by MSQE aggregation. */ public static final String NUM_GROUPS_LIMIT = "num_groups_limit"; + + /** Number of records that MSQE aggregation results, after sorting, should be limited to. + * Negative value disables trimming. */ + public static final String GROUP_TRIM_SIZE = "group_trim_size"; + public static final String MAX_INITIAL_RESULT_HOLDER_CAPACITY = "max_initial_result_holder_capacity"; } diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java index 141b20d422f7..42bd12433901 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/logical/PinotLogicalSortExchange.java @@ -34,7 +34,7 @@ /** * Pinot's implementation of {@code SortExchange} which needs information about whether to sort on the sender * and/or receiver side of the exchange. Every {@code Exchange} is broken into a send and a receive node and the - * decision on where to sort is made by the planner and this information has to b passed onto the send and receive + * decision on where to sort is made by the planner and this information has to be passed onto the send and receive * nodes for the correct execution. * * Note: This class does not extend {@code LogicalSortExchange} because its constructor which takes the list of diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java index a9ce6064b886..c431e18de9ae 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java @@ -24,12 +24,16 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.PriorityQueue; import javax.annotation.Nullable; +import org.apache.calcite.rel.RelFieldCollation; +import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.request.context.ExpressionContext; import org.apache.pinot.common.request.context.FunctionContext; import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.core.common.BlockValSet; import org.apache.pinot.core.operator.docvalsets.DataBlockValSet; import org.apache.pinot.core.operator.docvalsets.FilteredDataBlockValSet; @@ -39,10 +43,13 @@ import org.apache.pinot.core.query.aggregation.function.AggregationFunctionFactory; import org.apache.pinot.core.query.aggregation.function.CountAggregationFunction; import org.apache.pinot.core.util.DataBlockExtractUtils; +import org.apache.pinot.core.util.GroupByUtils; import org.apache.pinot.query.parser.CalciteRexExpressionParser; import org.apache.pinot.query.planner.logical.RexExpression; import org.apache.pinot.query.planner.plannode.AggregateNode; +import org.apache.pinot.query.planner.plannode.PlanNode; import org.apache.pinot.query.runtime.blocks.TransferableBlock; +import org.apache.pinot.query.runtime.operator.utils.SortUtils; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; @@ -50,11 +57,12 @@ /** - * AggregateOperator is used to aggregate values over a set of group by keys. + * AggregateOperator is used to aggregate values over a (potentially empty) set of group by keys in V2/MSQE. * Output data will be in the format of [group by key, aggregate result1, ... aggregate resultN] * When the list of aggregation calls is empty, this class is used to calculate distinct result based on group by keys. */ public class AggregateOperator extends MultiStageOperator { + private static final Logger LOGGER = LoggerFactory.getLogger(AggregateOperator.class); private static final String EXPLAIN_NAME = "AGGREGATE_OPERATOR"; private static final CountAggregationFunction COUNT_STAR_AGG_FUNCTION = @@ -64,12 +72,19 @@ public class AggregateOperator extends MultiStageOperator { private final DataSchema _resultSchema; private final MultistageAggregationExecutor _aggregationExecutor; private final MultistageGroupByExecutor _groupByExecutor; + @Nullable private TransferableBlock _eosBlock; private final StatMap _statMap = new StatMap<>(StatKey.class); private boolean _hasConstructedAggregateBlock; + private final boolean _errorOnNumGroupsLimit; + + // trimming - related members + private final int _groupTrimSize; + private final PriorityQueue _priorityQueue; + public AggregateOperator(OpChainExecutionContext context, MultiStageOperator input, AggregateNode node) { super(context); _input = input; @@ -88,8 +103,37 @@ public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inp maxFilterArgId = Math.max(maxFilterArgId, filterArgIds[i]); } - // Initialize the appropriate executor. List groupKeys = node.getGroupKeys(); + + //process order trimming hint + int groupTrimSize = getGroupTrimSize(node.getNodeHint()); + + if (groupTrimSize > -1) { + // limit is set to 0 if not pushed + int nodeLimit = node.getLimit() > 0 ? node.getLimit() : Integer.MAX_VALUE; + int limit = GroupByUtils.getTableCapacity(nodeLimit, groupTrimSize); + _groupTrimSize = limit; + if (limit == Integer.MAX_VALUE) { + // disable sorting because actual result can't realistically be bigger the limit + _priorityQueue = null; + } else { + List collations = node.getCollations(); + if (collations != null && collations.size() > 0) { + // order needs to be reversed so that peek() can be used to compare with each output row + _priorityQueue = + new PriorityQueue<>(groupTrimSize, new SortUtils.SortComparator(_resultSchema, collations, true)); + } else { + _priorityQueue = null; + } + } + } else { + _groupTrimSize = Integer.MAX_VALUE; + _priorityQueue = null; + } + + _errorOnNumGroupsLimit = getErrorOnNumGroupsLimit(context.getOpChainMetadata(), node.getNodeHint()); + + // Initialize the appropriate executor. AggregateNode.AggType aggType = node.getAggType(); // TODO: Allow leaf return final result for non-group-by queries boolean leafReturnFinalResult = node.isLeafReturnFinalResult(); @@ -105,6 +149,20 @@ public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inp } } + private int getGroupTrimSize(PlanNode.NodeHint nodeHint) { + if (nodeHint != null) { + Map options = nodeHint.getHintOptions().get(PinotHintOptions.AGGREGATE_HINT_OPTIONS); + if (options != null) { + String option = options.get(PinotHintOptions.AggregateOptions.GROUP_TRIM_SIZE); + if (option != null) { + return Integer.parseInt(option); + } + } + } + + return -1; + } + @Override public void registerExecution(long time, int numRows) { _statMap.merge(StatKey.EXECUTION_TIME_MS, time); @@ -152,14 +210,25 @@ private TransferableBlock produceAggregatedBlock() { if (_aggregationExecutor != null) { return new TransferableBlock(_aggregationExecutor.getResult(), _resultSchema, DataBlock.Type.ROW); } else { - List rows = _groupByExecutor.getResult(); + List rows; + if (_priorityQueue != null) { + rows = _groupByExecutor.getResult(_priorityQueue, _groupTrimSize); + } else { + rows = _groupByExecutor.getResult(_groupTrimSize); + } + if (rows.isEmpty()) { return _eosBlock; } else { TransferableBlock dataBlock = new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW); if (_groupByExecutor.isNumGroupsLimitReached()) { - _statMap.merge(StatKey.NUM_GROUPS_LIMIT_REACHED, true); - _input.earlyTerminate(); + if (_errorOnNumGroupsLimit) { + _input.earlyTerminate(); + throw new RuntimeException("NUM_GROUPS_LIMIT has been reached at " + _operatorId); + } else { + _statMap.merge(StatKey.NUM_GROUPS_LIMIT_REACHED, true); + _input.earlyTerminate(); + } } return dataBlock; } @@ -384,4 +453,18 @@ public StatMap.Type getType() { return _type; } } + + private boolean getErrorOnNumGroupsLimit(Map opChainMetadata, PlanNode.NodeHint nodeHint) { + if (nodeHint != null) { + Map options = nodeHint.getHintOptions().get(PinotHintOptions.AGGREGATE_HINT_OPTIONS); + if (options != null) { + String option = options.get(PinotHintOptions.AggregateOptions.ERROR_ON_NUM_GROUPS_LIMIT); + if (option != null) { + return Boolean.parseBoolean(option); + } + } + } + + return QueryOptionsUtils.getErrorOnNumGroupsLimit(opChainMetadata); + } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageAggregationExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageAggregationExecutor.java index d7503b558ebf..4597b8635435 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageAggregationExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageAggregationExecutor.java @@ -33,7 +33,8 @@ /** - * Class that executes all aggregation functions (without group-bys) for the multistage AggregateOperator. + * Class that executes all non-keyed aggregation functions (when there are no group by keys) for the multistage + * AggregateOperator. */ @SuppressWarnings({"rawtypes", "unchecked"}) public class MultistageAggregationExecutor { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java index 701f098182c9..e37798df0888 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.PriorityQueue; import javax.annotation.Nullable; import org.apache.pinot.calcite.rel.hint.PinotHintOptions; import org.apache.pinot.common.datablock.DataBlock; @@ -47,7 +48,7 @@ /** - * Class that executes the group by aggregations for the multistage AggregateOperator. + * Class that executes the keyed group by aggregations for the multistage AggregateOperator. */ @SuppressWarnings({"rawtypes", "unchecked"}) public class MultistageGroupByExecutor { @@ -69,9 +70,16 @@ public class MultistageGroupByExecutor { // because they use the zero based integer indexes to store results. private final GroupIdGenerator _groupIdGenerator; - public MultistageGroupByExecutor(int[] groupKeyIds, AggregationFunction[] aggFunctions, int[] filterArgIds, - int maxFilterArgId, AggType aggType, boolean leafReturnFinalResult, DataSchema resultSchema, - Map opChainMetadata, @Nullable PlanNode.NodeHint nodeHint) { + public MultistageGroupByExecutor( + int[] groupKeyIds, + AggregationFunction[] aggFunctions, + int[] filterArgIds, + int maxFilterArgId, + AggType aggType, + boolean leafReturnFinalResult, + DataSchema resultSchema, + Map opChainMetadata, + @Nullable PlanNode.NodeHint nodeHint) { _groupKeyIds = groupKeyIds; _aggFunctions = aggFunctions; _filterArgIds = filterArgIds; @@ -151,34 +159,84 @@ public void processBlock(TransferableBlock block) { } /** - * Fetches the result. + * Get aggregation result limited to first {@code maxRows} rows, ordered with {@code sortedRows} collection. */ - public List getResult() { - int numGroups = _groupIdGenerator.getNumGroups(); + public List getResult(PriorityQueue sortedRows, int maxRows) { + int numGroups = Math.min(_groupIdGenerator.getNumGroups(), maxRows); if (numGroups == 0) { return Collections.emptyList(); } - List rows = new ArrayList<>(numGroups); + int numKeys = _groupKeyIds.length; int numFunctions = _aggFunctions.length; ColumnDataType[] resultStoredTypes = _resultSchema.getStoredColumnDataTypes(); Iterator groupKeyIterator = _groupIdGenerator.getGroupKeyIterator(numKeys + numFunctions); + + int idx = 0; + while (idx++ < numGroups && groupKeyIterator.hasNext()) { + Object[] row = getRow(groupKeyIterator, numKeys, numFunctions, resultStoredTypes); + sortedRows.add(row); + } + while (groupKeyIterator.hasNext()) { - GroupIdGenerator.GroupKey groupKey = groupKeyIterator.next(); - int groupId = groupKey._groupId; - Object[] row = groupKey._row; - int columnId = numKeys; - for (int i = 0; i < numFunctions; i++) { - row[columnId++] = getResultValue(i, groupId); + // TODO: allocate new array row only if row enters set + Object[] row = getRow(groupKeyIterator, numKeys, numFunctions, resultStoredTypes); + if (sortedRows.comparator().compare(sortedRows.peek(), row) < 0) { + sortedRows.poll(); + sortedRows.offer(row); } - // Convert the results from AggregationFunction to the desired type - TypeUtils.convertRow(row, resultStoredTypes); + } + + int resultSize = sortedRows.size(); + ArrayList result = new ArrayList<>(sortedRows.size()); + for (int i = resultSize - 1; i >= 0; i--) { + result.add(sortedRows.poll()); + } + // reverse priority queue order because comparators are reversed + Collections.reverse(result); + return result; + } + + /** Get aggregation result limited to {@code maxRows} rows. */ + public List getResult(int trimSize) { + int numGroups = Math.min(_groupIdGenerator.getNumGroups(), trimSize); + if (numGroups == 0) { + return Collections.emptyList(); + } + + List rows = new ArrayList<>(numGroups); + int numKeys = _groupKeyIds.length; + int numFunctions = _aggFunctions.length; + ColumnDataType[] resultStoredTypes = _resultSchema.getStoredColumnDataTypes(); + Iterator groupKeyIterator = + _groupIdGenerator.getGroupKeyIterator(numKeys + numFunctions); + + int idx = 0; + while (groupKeyIterator.hasNext() && idx++ < numGroups) { + Object[] row = getRow(groupKeyIterator, numKeys, numFunctions, resultStoredTypes); rows.add(row); } return rows; } + private Object[] getRow( + Iterator groupKeyIterator, + int numKeys, + int numFunctions, + ColumnDataType[] resultStoredTypes) { + GroupIdGenerator.GroupKey groupKey = groupKeyIterator.next(); + int groupId = groupKey._groupId; + Object[] row = groupKey._row; + int columnId = numKeys; + for (int i = 0; i < numFunctions; i++) { + row[columnId++] = getResultValue(i, groupId); + } + // Convert the results from AggregationFunction to the desired type + TypeUtils.convertRow(row, resultStoredTypes); + return row; + } + private Object getResultValue(int functionId, int groupId) { AggregationFunction aggFunction = _aggFunctions[functionId]; switch (_aggType) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java index 41d246858210..40c298b99a88 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/server/ServerPlanRequestUtils.java @@ -76,8 +76,12 @@ private ServerPlanRequestUtils() { new ArrayList<>(QueryRewriterFactory.getQueryRewriters(QUERY_REWRITERS_CLASS_NAMES)); private static final QueryOptimizer QUERY_OPTIMIZER = new QueryOptimizer(); - public static OpChain compileLeafStage(OpChainExecutionContext executionContext, StagePlan stagePlan, - HelixManager helixManager, ServerMetrics serverMetrics, QueryExecutor leafQueryExecutor, + public static OpChain compileLeafStage( + OpChainExecutionContext executionContext, + StagePlan stagePlan, + HelixManager helixManager, + ServerMetrics serverMetrics, + QueryExecutor leafQueryExecutor, ExecutorService executorService) { return compileLeafStage(executionContext, stagePlan, helixManager, serverMetrics, leafQueryExecutor, executorService, (planNode, multiStageOperator) -> { @@ -91,9 +95,14 @@ public static OpChain compileLeafStage(OpChainExecutionContext executionContext, * @param stagePlan the distribute stage plan on the leaf. * @return an opChain that executes the leaf-stage, with the leaf-stage execution encapsulated within. */ - public static OpChain compileLeafStage(OpChainExecutionContext executionContext, StagePlan stagePlan, - HelixManager helixManager, ServerMetrics serverMetrics, QueryExecutor leafQueryExecutor, - ExecutorService executorService, BiConsumer relationConsumer, boolean explain) { + public static OpChain compileLeafStage(OpChainExecutionContext executionContext, + StagePlan stagePlan, + HelixManager helixManager, + ServerMetrics serverMetrics, + QueryExecutor leafQueryExecutor, + ExecutorService executorService, + BiConsumer relationConsumer, + boolean explain) { long queryArrivalTimeMs = System.currentTimeMillis(); ServerPlanRequestContext serverContext = new ServerPlanRequestContext(stagePlan, leafQueryExecutor, executorService, executionContext.getPipelineBreakerResult()); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 5a4ce98286c7..253f800d5d04 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -390,43 +390,51 @@ private TimeSeriesDispatchClient getOrCreateTimeSeriesDispatchClient( return _timeSeriesDispatchClientMap.computeIfAbsent(key, k -> new TimeSeriesDispatchClient(hostname, port)); } + // There is no reduction happening here, results are simply concatenated. @VisibleForTesting - public static QueryResult runReducer(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, - Map queryOptions, MailboxService mailboxService) { + public static QueryResult runReducer(long requestId, + DispatchableSubPlan subPlan, + long timeoutMs, + Map queryOptions, + MailboxService mailboxService) { + long startTimeMs = System.currentTimeMillis(); long deadlineMs = startTimeMs + timeoutMs; - // NOTE: Reduce stage is always stage 0 - DispatchablePlanFragment dispatchableStagePlan = dispatchableSubPlan.getQueryStageList().get(0); - PlanFragment planFragment = dispatchableStagePlan.getPlanFragment(); + DispatchablePlanFragment stagePlan = subPlan.getQueryStageList().get(0); + PlanFragment planFragment = stagePlan.getPlanFragment(); PlanNode rootNode = planFragment.getFragmentRoot(); + Preconditions.checkState(rootNode instanceof MailboxReceiveNode, "Expecting mailbox receive node as root of reduce stage, got: %s", rootNode.getClass().getSimpleName()); + MailboxReceiveNode receiveNode = (MailboxReceiveNode) rootNode; - List workerMetadataList = dispatchableStagePlan.getWorkerMetadataList(); - Preconditions.checkState(workerMetadataList.size() == 1, "Expecting single worker for reduce stage, got: %s", - workerMetadataList.size()); - StageMetadata stageMetadata = new StageMetadata(0, workerMetadataList, dispatchableStagePlan.getCustomProperties()); + List workerMetadata = stagePlan.getWorkerMetadataList(); + + Preconditions.checkState(workerMetadata.size() == 1, + "Expecting single worker for reduce stage, got: %s", workerMetadata.size()); + + StageMetadata stageMetadata = new StageMetadata(0, workerMetadata, stagePlan.getCustomProperties()); ThreadExecutionContext parentContext = Tracing.getThreadAccountant().getThreadExecutionContext(); - OpChainExecutionContext opChainExecutionContext = + OpChainExecutionContext executionContext = new OpChainExecutionContext(mailboxService, requestId, deadlineMs, queryOptions, stageMetadata, - workerMetadataList.get(0), null, parentContext); + workerMetadata.get(0), null, parentContext); - PairList resultFields = dispatchableSubPlan.getQueryResultFields(); - DataSchema sourceDataSchema = receiveNode.getDataSchema(); + PairList resultFields = subPlan.getQueryResultFields(); + DataSchema sourceSchema = receiveNode.getDataSchema(); int numColumns = resultFields.size(); String[] columnNames = new String[numColumns]; ColumnDataType[] columnTypes = new ColumnDataType[numColumns]; for (int i = 0; i < numColumns; i++) { Map.Entry field = resultFields.get(i); columnNames[i] = field.getValue(); - columnTypes[i] = sourceDataSchema.getColumnDataType(field.getKey()); + columnTypes[i] = sourceSchema.getColumnDataType(field.getKey()); } - DataSchema resultDataSchema = new DataSchema(columnNames, columnTypes); + DataSchema resultSchema = new DataSchema(columnNames, columnTypes); ArrayList resultRows = new ArrayList<>(); TransferableBlock block; - try (MailboxReceiveOperator receiveOperator = new MailboxReceiveOperator(opChainExecutionContext, receiveNode)) { + try (MailboxReceiveOperator receiveOperator = new MailboxReceiveOperator(executionContext, receiveNode)) { block = receiveOperator.nextBlock(); while (!TransferableBlockUtils.isEndOfStream(block)) { DataBlock dataBlock = block.getDataBlock(); @@ -456,7 +464,7 @@ public static QueryResult runReducer(long requestId, DispatchableSubPlan dispatc assert block.isSuccessfulEndOfStreamBlock(); MultiStageQueryStats queryStats = block.getQueryStats(); assert queryStats != null; - return new QueryResult(new ResultTable(resultDataSchema, resultRows), queryStats, + return new QueryResult(new ResultTable(resultSchema, resultRows), queryStats, System.currentTimeMillis() - startTimeMs); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 8e27bbccef35..76fd040e627e 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -445,6 +445,9 @@ public static class QueryOptionKey { public static final String ORDER_BY_ALGORITHM = "orderByAlgorithm"; public static final String MULTI_STAGE_LEAF_LIMIT = "multiStageLeafLimit"; + + /** Throw an exception on reaching num_groups_limit instead of just setting a flag. */ + public static final String ERROR_ON_NUM_GROUPS_LIMIT = "errorOnNumGroupsLimit"; public static final String NUM_GROUPS_LIMIT = "numGroupsLimit"; public static final String MAX_INITIAL_RESULT_HOLDER_CAPACITY = "maxInitialResultHolderCapacity"; public static final String MIN_INITIAL_INDEXED_TABLE_CAPACITY = "minInitialIndexedTableCapacity"; diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java index da83dc219419..25415c7b5671 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java @@ -429,6 +429,10 @@ public String forDeleteTableWithType(String tableName, String tableType) { return StringUtil.join("/", _baseUrl, "tables", tableName + "?type=" + tableType); } + public String forServersToSegmentsMap(String tableName, String tableType) { + return StringUtil.join("/", _baseUrl, "segments", tableName, "servers?type=" + tableType); + } + public String forSegmentListAPI(String tableName) { return forSegmentListAPI(tableName, null, false, Long.MIN_VALUE, Long.MAX_VALUE, false); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java index 5e9d915cfc46..5151eede96f8 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/TableConfigBuilder.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.base.Preconditions; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -145,6 +146,14 @@ public TableConfigBuilder setIsDimTable(boolean isDimTable) { return this; } + public TableConfigBuilder addFieldConfig(FieldConfig config) { + if (_fieldConfigList == null) { + _fieldConfigList = new ArrayList<>(); + } + _fieldConfigList.add(config); + return this; + } + @Deprecated public TableConfigBuilder setLLC(boolean isLLC) { Preconditions.checkState(_tableType == TableType.REALTIME); From c75da16f038a83655ae0f08022d666bb7623bf92 Mon Sep 17 00:00:00 2001 From: Bolek Ziobrowski <26925920+bziobrowski@users.noreply.github.com> Date: Tue, 31 Dec 2024 17:08:09 +0100 Subject: [PATCH 3/6] Add group_trim_size query option and executor setting. --- .../utils/config/QueryOptionsUtils.java | 7 +++ .../plan/maker/InstancePlanMakerImplV2.java | 3 ++ .../calcite/rel/hint/PinotHintOptions.java | 6 +-- .../pinot/query/runtime/QueryRunner.java | 18 +++++++ .../runtime/operator/AggregateOperator.java | 16 +++++-- .../operator/AggregateOperatorTest.java | 47 +++++++++++++++++++ .../runtime/operator/OperatorTestUtil.java | 4 ++ .../pinot/spi/utils/CommonConstants.java | 14 ++++++ 8 files changed, 108 insertions(+), 7 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java index e29da5cc6d08..5f88a9691c0b 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java @@ -213,6 +213,13 @@ public static Integer getMaxExecutionThreads(Map queryOptions) { return checkedParseIntPositive(QueryOptionKey.MAX_EXECUTION_THREADS, maxExecutionThreadsString); } + @Nullable + public static Integer getGroupTrimSize(Map queryOptions) { + String groupTrimSize = queryOptions.get(QueryOptionKey.GROUP_TRIM_SIZE); + // NOTE: Non-positive value means turning off the intermediate level trim + return uncheckedParseInt(QueryOptionKey.GROUP_TRIM_SIZE, groupTrimSize); + } + @Nullable public static Integer getMinSegmentGroupTrimSize(Map queryOptions) { String minSegmentGroupTrimSizeString = queryOptions.get(QueryOptionKey.MIN_SEGMENT_GROUP_TRIM_SIZE); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java index cadce4bcf6d0..0475c62c1711 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/maker/InstancePlanMakerImplV2.java @@ -76,6 +76,9 @@ public class InstancePlanMakerImplV2 implements PlanMaker { public static final String NUM_GROUPS_LIMIT_KEY = "num.groups.limit"; public static final int DEFAULT_NUM_GROUPS_LIMIT = 100_000; + // By default, group trimming in AggregateOperator is disabled + public static final int DEFAULT_GROUP_TRIM_SIZE = -1; + // Instance config key for minimum segment-level group trim size // Set as pinot.server.query.executor.min.segment.group.trim.size public static final String MIN_SEGMENT_GROUP_TRIM_SIZE_KEY = "min.segment.group.trim.size"; diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java index 1ea364179574..d98b86a0f760 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/rel/hint/PinotHintOptions.java @@ -43,10 +43,10 @@ public static class AggregateOptions { public static final String IS_PARTITIONED_BY_GROUP_BY_KEYS = "is_partitioned_by_group_by_keys"; public static final String IS_LEAF_RETURN_FINAL_RESULT = "is_leaf_return_final_result"; public static final String IS_SKIP_LEAF_STAGE_GROUP_BY = "is_skip_leaf_stage_group_by"; - public static final String IS_ENABLE_GROUP_TRIM = "is_enable_group_trim"; - /** Enables trimming of aggregation intermediate results by pushing down order by and limit to leaf stage. */ - public static final String ENABLE_GROUP_TRIM = "is_enable_group_trim"; + /** Enables trimming of aggregation intermediate results by pushing down order by and limit, + * down to leaf stage if possible. */ + public static final String IS_ENABLE_GROUP_TRIM = "is_enable_group_trim"; /** Throw an exception on reaching num_groups_limit instead of just setting a flag. */ public static final String ERROR_ON_NUM_GROUPS_LIMIT = "error_on_num_groups_limit"; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index ac335a1674c4..1bb47a8f009d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -108,6 +108,9 @@ public class QueryRunner { // Group-by settings @Nullable private Integer _numGroupsLimit; + @Nullable + private Integer _groupTrimSize; + @Nullable private Integer _maxInitialResultHolderCapacity; @Nullable @@ -141,16 +144,23 @@ public void init(PinotConfiguration config, InstanceDataManager instanceDataMana // TODO: Consider using separate config for intermediate stage and leaf stage String numGroupsLimitStr = config.getProperty(CommonConstants.Server.CONFIG_OF_QUERY_EXECUTOR_NUM_GROUPS_LIMIT); _numGroupsLimit = numGroupsLimitStr != null ? Integer.parseInt(numGroupsLimitStr) : null; + + String groupTrimSizeStr = config.getProperty(CommonConstants.Server.CONFIG_OF_QUERY_EXECUTOR_GROUP_TRIM_SIZE); + _groupTrimSize = groupTrimSizeStr != null ? Integer.parseInt(groupTrimSizeStr) : null; + String maxInitialGroupHolderCapacity = config.getProperty(CommonConstants.Server.CONFIG_OF_QUERY_EXECUTOR_MAX_INITIAL_RESULT_HOLDER_CAPACITY); _maxInitialResultHolderCapacity = maxInitialGroupHolderCapacity != null ? Integer.parseInt(maxInitialGroupHolderCapacity) : null; + String minInitialIndexedTableCapacityStr = config.getProperty(CommonConstants.Server.CONFIG_OF_QUERY_EXECUTOR_MIN_INITIAL_INDEXED_TABLE_CAPACITY); _minInitialIndexedTableCapacity = minInitialIndexedTableCapacityStr != null ? Integer.parseInt(minInitialIndexedTableCapacityStr) : null; + String maxRowsInJoinStr = config.getProperty(CommonConstants.MultiStageQueryRunner.KEY_OF_MAX_ROWS_IN_JOIN); _maxRowsInJoin = maxRowsInJoinStr != null ? Integer.parseInt(maxRowsInJoinStr) : null; + String joinOverflowModeStr = config.getProperty(CommonConstants.MultiStageQueryRunner.KEY_OF_JOIN_OVERFLOW_MODE); _joinOverflowMode = joinOverflowModeStr != null ? JoinOverFlowMode.valueOf(joinOverflowModeStr) : null; @@ -337,6 +347,14 @@ private Map consolidateMetadata(Map customProper opChainMetadata.put(QueryOptionKey.NUM_GROUPS_LIMIT, Integer.toString(numGroupsLimit)); } + Integer groupTrimSize = QueryOptionsUtils.getGroupTrimSize(opChainMetadata); + if (groupTrimSize == null) { + groupTrimSize = _groupTrimSize; + } + if (groupTrimSize != null) { + opChainMetadata.put(QueryOptionKey.GROUP_TRIM_SIZE, Integer.toString(groupTrimSize)); + } + Integer maxInitialResultHolderCapacity = QueryOptionsUtils.getMaxInitialResultHolderCapacity(opChainMetadata); if (maxInitialResultHolderCapacity == null) { maxInitialResultHolderCapacity = _maxInitialResultHolderCapacity; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java index c431e18de9ae..e8173534b200 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.query.runtime.operator; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Collections; @@ -39,6 +40,7 @@ import org.apache.pinot.core.operator.docvalsets.FilteredDataBlockValSet; import org.apache.pinot.core.operator.docvalsets.FilteredRowBasedBlockValSet; import org.apache.pinot.core.operator.docvalsets.RowBasedBlockValSet; +import org.apache.pinot.core.plan.maker.InstancePlanMakerImplV2; import org.apache.pinot.core.query.aggregation.function.AggregationFunction; import org.apache.pinot.core.query.aggregation.function.AggregationFunctionFactory; import org.apache.pinot.core.query.aggregation.function.CountAggregationFunction; @@ -106,7 +108,7 @@ public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inp List groupKeys = node.getGroupKeys(); //process order trimming hint - int groupTrimSize = getGroupTrimSize(node.getNodeHint()); + int groupTrimSize = getGroupTrimSize(node.getNodeHint(), context.getOpChainMetadata()); if (groupTrimSize > -1) { // limit is set to 0 if not pushed @@ -118,7 +120,7 @@ public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inp _priorityQueue = null; } else { List collations = node.getCollations(); - if (collations != null && collations.size() > 0) { + if (collations != null && !collations.isEmpty()) { // order needs to be reversed so that peek() can be used to compare with each output row _priorityQueue = new PriorityQueue<>(groupTrimSize, new SortUtils.SortComparator(_resultSchema, collations, true)); @@ -149,7 +151,7 @@ public AggregateOperator(OpChainExecutionContext context, MultiStageOperator inp } } - private int getGroupTrimSize(PlanNode.NodeHint nodeHint) { + private int getGroupTrimSize(PlanNode.NodeHint nodeHint, Map opChainMetadata) { if (nodeHint != null) { Map options = nodeHint.getHintOptions().get(PinotHintOptions.AGGREGATE_HINT_OPTIONS); if (options != null) { @@ -160,7 +162,8 @@ private int getGroupTrimSize(PlanNode.NodeHint nodeHint) { } } - return -1; + Integer groupTrimSize = QueryOptionsUtils.getGroupTrimSize(opChainMetadata); + return groupTrimSize != null ? groupTrimSize : InstancePlanMakerImplV2.DEFAULT_GROUP_TRIM_SIZE; } @Override @@ -467,4 +470,9 @@ private boolean getErrorOnNumGroupsLimit(Map opChainMetadata, Pl return QueryOptionsUtils.getErrorOnNumGroupsLimit(opChainMetadata); } + + @VisibleForTesting + int getGroupTrimSize() { + return _groupTrimSize; + } } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java index b2e73f226a3a..56a83cb36e8b 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/AggregateOperatorTest.java @@ -33,7 +33,10 @@ import org.apache.pinot.query.runtime.blocks.TransferableBlock; import org.apache.pinot.query.runtime.blocks.TransferableBlockTestUtils; import org.apache.pinot.query.runtime.blocks.TransferableBlockUtils; +import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.apache.pinot.spi.utils.CommonConstants; import org.mockito.Mock; +import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -265,6 +268,50 @@ public void shouldHandleGroupLimitExceed() { "num groups limit should be reached"); } + @Test + public void testGroupTrimSizeIsDisabledByDefault() { + PlanNode.NodeHint nodeHint = null; + OpChainExecutionContext context = OperatorTestUtil.getTracingContext(); + + Assert.assertEquals(getAggregateOperator(context, nodeHint, 10).getGroupTrimSize(), Integer.MAX_VALUE); + Assert.assertEquals(getAggregateOperator(context, nodeHint, 0).getGroupTrimSize(), Integer.MAX_VALUE); + } + + @Test + public void testGroupTrimSizeDependsOnContextValue() { + PlanNode.NodeHint nodeHint = null; + OpChainExecutionContext context = + OperatorTestUtil.getContext(Map.of(CommonConstants.Broker.Request.QueryOptionKey.GROUP_TRIM_SIZE, "100")); + + AggregateOperator operator = getAggregateOperator(context, nodeHint, 5); + + Assert.assertEquals(operator.getGroupTrimSize(), 100); + } + + @Test + public void testGroupTrimHintOverridesContextValue() { + PlanNode.NodeHint nodeHint = new PlanNode.NodeHint(Map.of(PinotHintOptions.AGGREGATE_HINT_OPTIONS, + Map.of(PinotHintOptions.AggregateOptions.GROUP_TRIM_SIZE, "30"))); + + OpChainExecutionContext context = + OperatorTestUtil.getContext(Map.of(CommonConstants.Broker.Request.QueryOptionKey.GROUP_TRIM_SIZE, "100")); + + AggregateOperator operator = getAggregateOperator(context, nodeHint, 5); + + Assert.assertEquals(operator.getGroupTrimSize(), 30); + } + + private AggregateOperator getAggregateOperator(OpChainExecutionContext context, PlanNode.NodeHint nodeHint, + int limit) { + List aggCalls = List.of(getSum(new RexExpression.InputRef(1))); + List filterArgs = List.of(-1); + List groupKeys = List.of(0); + DataSchema resultSchema = new DataSchema(new String[]{"group", "sum"}, new ColumnDataType[]{INT, DOUBLE}); + return new AggregateOperator(context, _input, + new AggregateNode(-1, resultSchema, nodeHint, List.of(), aggCalls, filterArgs, groupKeys, AggType.DIRECT, + false, null, limit)); + } + private static RexExpression.FunctionCall getSum(RexExpression arg) { return new RexExpression.FunctionCall(ColumnDataType.INT, SqlKind.SUM.name(), List.of(arg)); } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java index f279e5992b14..0d6317ab2d53 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTestUtil.java @@ -90,6 +90,10 @@ public static OpChainExecutionContext getTracingContext() { return getTracingContext(ImmutableMap.of(CommonConstants.Broker.Request.TRACE, "true")); } + public static OpChainExecutionContext getContext(Map opChainMetadata) { + return getTracingContext(opChainMetadata); + } + public static OpChainExecutionContext getNoTracingContext() { return getTracingContext(ImmutableMap.of()); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 76fd040e627e..93f5b03a0ceb 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -409,9 +409,21 @@ public static class QueryOptionKey { public static final String ROUTING_OPTIONS = "routingOptions"; public static final String USE_SCAN_REORDER_OPTIMIZATION = "useScanReorderOpt"; public static final String MAX_EXECUTION_THREADS = "maxExecutionThreads"; + + /** Number of groups AggregateOperator should limit result to after sorting. + * Trimming happens only when (sub)query contains order by and limit clause. */ + public static final String GROUP_TRIM_SIZE = "groupTrimSize"; + + /** Number of groups GroupByOperator should limit result to after sorting. + * Trimming happens only when (sub)query contains order by clause. */ public static final String MIN_SEGMENT_GROUP_TRIM_SIZE = "minSegmentGroupTrimSize"; + + /** Max number of groups GroupByCombineOperator (running at server) should return .*/ public static final String MIN_SERVER_GROUP_TRIM_SIZE = "minServerGroupTrimSize"; + + /** Max number of groups GroupByDataTableReducer (running at broker) should return. */ public static final String MIN_BROKER_GROUP_TRIM_SIZE = "minBrokerGroupTrimSize"; + public static final String NUM_REPLICA_GROUPS_TO_QUERY = "numReplicaGroupsToQuery"; public static final String USE_FIXED_REPLICA = "useFixedReplica"; public static final String EXPLAIN_PLAN_VERBOSE = "explainPlanVerbose"; @@ -702,6 +714,8 @@ public static class Server { public static final String CONFIG_OF_QUERY_EXECUTOR_TIMEOUT = "pinot.server.query.executor.timeout"; public static final String CONFIG_OF_QUERY_EXECUTOR_NUM_GROUPS_LIMIT = "pinot.server.query.executor.num.groups.limit"; + public static final String CONFIG_OF_QUERY_EXECUTOR_GROUP_TRIM_SIZE = + "pinot.server.query.executor.group.trim.size"; public static final String CONFIG_OF_QUERY_EXECUTOR_MAX_INITIAL_RESULT_HOLDER_CAPACITY = "pinot.server.query.executor.max.init.group.holder.capacity"; public static final String CONFIG_OF_QUERY_EXECUTOR_MIN_INITIAL_INDEXED_TABLE_CAPACITY = From 017a1bc920949ebf1ce185f51fd0a117749b0e39 Mon Sep 17 00:00:00 2001 From: Bolek Ziobrowski <26925920+bziobrowski@users.noreply.github.com> Date: Wed, 8 Jan 2025 18:01:05 +0100 Subject: [PATCH 4/6] Applied CR suggestions. --- .../query/reduce/BrokerReduceService.java | 78 ++++++------ .../tests/GroupByOptionsIntegrationTest.java | 118 +++--------------- .../runtime/operator/AggregateOperator.java | 3 +- .../operator/MultistageGroupByExecutor.java | 19 ++- 4 files changed, 64 insertions(+), 154 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index ccfe789fe52d..3bf174e7c648 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -58,12 +58,9 @@ public BrokerReduceService(PinotConfiguration config) { super(config); } - public BrokerResponseNative reduceOnDataTable( - BrokerRequest brokerRequest, - BrokerRequest serverBrokerRequest, - Map dataTableMap, - long reduceTimeOutMs, - BrokerMetrics brokerMetrics) { + public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, BrokerRequest serverBrokerRequest, + Map dataTableMap, + long reduceTimeOutMs, BrokerMetrics brokerMetrics) { if (dataTableMap.isEmpty()) { // Empty response. return BrokerResponseNative.empty(); @@ -74,12 +71,12 @@ public BrokerResponseNative reduceOnDataTable( queryOptions != null && Boolean.parseBoolean(queryOptions.get(CommonConstants.Broker.Request.TRACE)); ExecutionStatsAggregator aggregator = new ExecutionStatsAggregator(enableTrace); - BrokerResponseNative response = new BrokerResponseNative(); + BrokerResponseNative brokerResponseNative = new BrokerResponseNative(); // Cache a data schema from data tables (try to cache one with data rows associated with it). - DataSchema schemaOfEmptyTable = null; - DataSchema schemaOfNonEmptyTable = null; - List serversWithConflictingSchema = new ArrayList<>(); + DataSchema dataSchemaFromEmptyDataTable = null; + DataSchema dataSchemaFromNonEmptyDataTable = null; + List serversWithConflictingDataSchema = new ArrayList<>(); // Process server response metadata. Iterator> iterator = dataTableMap.entrySet().iterator(); @@ -97,19 +94,19 @@ public BrokerResponseNative reduceOnDataTable( } else { // Try to cache a data table with data rows inside, or cache one with data schema inside. if (dataTable.getNumberOfRows() == 0) { - if (schemaOfEmptyTable == null) { - schemaOfEmptyTable = dataSchema; + if (dataSchemaFromEmptyDataTable == null) { + dataSchemaFromEmptyDataTable = dataSchema; } iterator.remove(); } else { - if (schemaOfNonEmptyTable == null) { - schemaOfNonEmptyTable = dataSchema; + if (dataSchemaFromNonEmptyDataTable == null) { + dataSchemaFromNonEmptyDataTable = dataSchema; } else { // Remove data tables with conflicting data schema. // NOTE: Only compare the column data types, since the column names (string representation of expression) // can change across different versions. - if (!Arrays.equals(dataSchema.getColumnDataTypes(), schemaOfNonEmptyTable.getColumnDataTypes())) { - serversWithConflictingSchema.add(entry.getKey()); + if (!Arrays.equals(dataSchema.getColumnDataTypes(), dataSchemaFromNonEmptyDataTable.getColumnDataTypes())) { + serversWithConflictingDataSchema.add(entry.getKey()); iterator.remove(); } } @@ -121,51 +118,52 @@ public BrokerResponseNative reduceOnDataTable( String rawTableName = TableNameBuilder.extractRawTableName(tableName); // Set execution statistics and Update broker metrics. - aggregator.setStats(rawTableName, response, brokerMetrics); + aggregator.setStats(rawTableName, brokerResponseNative, brokerMetrics); // Report the servers with conflicting data schema. - if (!serversWithConflictingSchema.isEmpty()) { + if (!serversWithConflictingDataSchema.isEmpty()) { String errorMessage = QueryException.MERGE_RESPONSE_ERROR.getMessage() + ": responses for table: " + tableName - + " from servers: " + serversWithConflictingSchema + " got dropped due to data schema inconsistency."; + + " from servers: " + serversWithConflictingDataSchema + " got dropped due to data schema inconsistency."; LOGGER.warn(errorMessage); brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.RESPONSE_MERGE_EXCEPTIONS, 1); - response.addException(new QueryProcessingException(QueryException.MERGE_RESPONSE_ERROR_CODE, errorMessage)); + brokerResponseNative.addException( + new QueryProcessingException(QueryException.MERGE_RESPONSE_ERROR_CODE, errorMessage)); } // NOTE: When there is no cached data schema, that means all servers encountered exception. In such case, return the // response with metadata only. DataSchema cachedDataSchema = - schemaOfNonEmptyTable != null ? schemaOfNonEmptyTable : schemaOfEmptyTable; + dataSchemaFromNonEmptyDataTable != null ? dataSchemaFromNonEmptyDataTable : dataSchemaFromEmptyDataTable; if (cachedDataSchema == null) { - return response; + return brokerResponseNative; } - Integer minGroupTrimSizeOpt = null; - Integer groupTrimThresholdOpt = null; - Integer minInitialIndexedTableCapOpt = null; + QueryContext serverQueryContext = QueryContextConverterUtils.getQueryContext(serverBrokerRequest.getPinotQuery()); + DataTableReducer dataTableReducer = ResultReducerFactory.getResultReducer(serverQueryContext); + Integer minGroupTrimSizeQueryOption = null; + Integer groupTrimThresholdQueryOption = null; + Integer minInitialIndexedTableCapacityQueryOption = null; if (queryOptions != null) { - minGroupTrimSizeOpt = QueryOptionsUtils.getMinBrokerGroupTrimSize(queryOptions); - groupTrimThresholdOpt = QueryOptionsUtils.getGroupTrimThreshold(queryOptions); - minInitialIndexedTableCapOpt = QueryOptionsUtils.getMinInitialIndexedTableCapacity(queryOptions); + minGroupTrimSizeQueryOption = QueryOptionsUtils.getMinBrokerGroupTrimSize(queryOptions); + groupTrimThresholdQueryOption = QueryOptionsUtils.getGroupTrimThreshold(queryOptions); + minInitialIndexedTableCapacityQueryOption = QueryOptionsUtils.getMinInitialIndexedTableCapacity(queryOptions); } - - int minGroupTrimSize = minGroupTrimSizeOpt != null ? minGroupTrimSizeOpt : _minGroupTrimSize; - int groupTrimThreshold = groupTrimThresholdOpt != null ? groupTrimThresholdOpt : _groupByTrimThreshold; + int minGroupTrimSize = minGroupTrimSizeQueryOption != null ? minGroupTrimSizeQueryOption : _minGroupTrimSize; + int groupTrimThreshold = + groupTrimThresholdQueryOption != null ? groupTrimThresholdQueryOption : _groupByTrimThreshold; int minInitialIndexedTableCapacity = - minInitialIndexedTableCapOpt != null ? minInitialIndexedTableCapOpt : _minInitialIndexedTableCapacity; + minInitialIndexedTableCapacityQueryOption != null ? minInitialIndexedTableCapacityQueryOption + : _minInitialIndexedTableCapacity; - QueryContext serverQueryContext = QueryContextConverterUtils.getQueryContext(serverBrokerRequest.getPinotQuery()); try { - DataTableReducer reducer = ResultReducerFactory.getResultReducer(serverQueryContext); - reducer.reduceAndSetResults(rawTableName, cachedDataSchema, dataTableMap, response, + dataTableReducer.reduceAndSetResults(rawTableName, cachedDataSchema, dataTableMap, brokerResponseNative, new DataTableReducerContext(_reduceExecutorService, _maxReduceThreadsPerQuery, reduceTimeOutMs, groupTrimThreshold, minGroupTrimSize, minInitialIndexedTableCapacity), brokerMetrics); } catch (EarlyTerminationException e) { - response.addException( + brokerResponseNative.addException( new QueryProcessingException(QueryException.QUERY_CANCELLATION_ERROR_CODE, e.toString())); } - QueryContext queryContext; if (brokerRequest == serverBrokerRequest) { queryContext = serverQueryContext; @@ -176,13 +174,13 @@ public BrokerResponseNative reduceOnDataTable( throw new BadQueryRequestException("Nested query is not supported without gapfill"); } BaseGapfillProcessor gapfillProcessor = GapfillProcessorFactory.getGapfillProcessor(queryContext, gapfillType); - gapfillProcessor.process(response); + gapfillProcessor.process(brokerResponseNative); } if (!serverQueryContext.isExplain()) { - updateAlias(queryContext, response); + updateAlias(queryContext, brokerResponseNative); } - return response; + return brokerResponseNative; } public void shutDown() { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java index 715a54a84b9e..03af87b0602f 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/GroupByOptionsIntegrationTest.java @@ -431,8 +431,8 @@ public void assertResultAndPlan(String option, String query, String expectedResu JsonNode result = postV2Query(sql); JsonNode plan = postV2Query(option + " set explainAskingServers=true; explain plan for " + query); - Assert.assertEquals(toResultStr(result), new CharSeq(expectedResult)); - Assert.assertEquals(toExplainStr(plan), new CharSeq(expectedPlan)); + Assert.assertEquals(toResultStr(result), expectedResult); + Assert.assertEquals(toExplainStr(plan), expectedPlan); } @Test @@ -488,11 +488,11 @@ private void assertNumGroupsLimitException(String query) throws Exception { JsonNode result = postV2Query(query); - CharSeq errorMessage = toResultStr(result); + String errorMessage = toResultStr(result); Assert.assertTrue(errorMessage.startsWith("QueryExecutionError:\n" + "Received error query execution result block: {1000=NUM_GROUPS_LIMIT has been reached at "), - errorMessage.toString()); + errorMessage); } // for debug only @@ -511,9 +511,9 @@ private JsonNode postV2Query(String query) getExtraQueryProperties()); } - private static @NotNull CharSeq toResultStr(JsonNode mainNode) { + private static @NotNull String toResultStr(JsonNode mainNode) { if (mainNode == null) { - return new CharSeq(new StringBuilder("null")); + return "null"; } JsonNode node = mainNode.get(RESULT_TABLE); if (node == null) { @@ -522,9 +522,9 @@ private JsonNode postV2Query(String query) return toString(node); } - private static @NotNull CharSeq toExplainStr(JsonNode mainNode) { + private static @NotNull String toExplainStr(JsonNode mainNode) { if (mainNode == null) { - return new CharSeq(new StringBuilder("null")); + return "null"; } JsonNode node = mainNode.get(RESULT_TABLE); if (node == null) { @@ -533,101 +533,15 @@ private JsonNode postV2Query(String query) return toExplainString(node); } - static class CharSeq implements CharSequence { - private final StringBuilder _sb; - - CharSeq(StringBuilder sb) { - _sb = sb; - } - - CharSeq(String s) { - _sb = new StringBuilder(s); - } - - @Override - public int length() { - return _sb.length(); - } - - @Override - public char charAt(int index) { - return _sb.charAt(index); - } - - public boolean startsWith(CharSequence cs) { - if (cs.length() > _sb.length()) { - return false; - } - - for (int i = 0, len = cs.length(); i < len; i++) { - if (_sb.charAt(i) != cs.charAt(i)) { - return false; - } - } - - return true; - } - - @Override - public @NotNull CharSequence subSequence(int start, int end) { - return new CharSeq(_sb.substring(start, end)); - } - - @NotNull - @Override - public String toString() { - return _sb.toString(); - } - - @Override - public int hashCode() { - int hc = 0; - for (int i = 0, len = _sb.length(); i < len; i++) { - hc = 31 * hc + _sb.charAt(i); - } - return hc; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (!(obj instanceof CharSequence)) { - return false; - } - - CharSequence other = (CharSequence) obj; - if (_sb.length() != other.length()) { - return false; - } - - for (int i = 0, len = _sb.length(); i < len; i++) { - if (_sb.charAt(i) != other.charAt(i)) { - return false; - } - } - - return true; - } - - CharSeq append(CharSequence other) { - _sb.append(other); - return this; - } - } - - public static CharSeq toErrorString(JsonNode node) { - StringBuilder buf = new StringBuilder(); + public static String toErrorString(JsonNode node) { JsonNode jsonNode = node.get(0); if (jsonNode != null) { - buf.append(jsonNode.get("message").textValue()); + return jsonNode.get("message").textValue(); } - return new CharSeq(buf); + return ""; } - public static CharSeq toString(JsonNode node) { + public static String toString(JsonNode node) { StringBuilder buf = new StringBuilder(); ArrayNode columnNames = (ArrayNode) node.get("dataSchema").get("columnNames"); ArrayNode columnTypes = (ArrayNode) node.get("dataSchema").get("columnDataTypes"); @@ -657,13 +571,11 @@ public static CharSeq toString(JsonNode node) { } } - return new CharSeq(buf); + return buf.toString(); } - public static CharSeq toExplainString(JsonNode node) { - StringBuilder buf = new StringBuilder(); - buf.append(node.get("rows").get(0).get(1).textValue()); - return new CharSeq(buf); + public static String toExplainString(JsonNode node) { + return node.get("rows").get(0).get(1).textValue(); } @AfterClass diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java index e8173534b200..ea5e950dc4ab 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java @@ -59,7 +59,7 @@ /** - * AggregateOperator is used to aggregate values over a (potentially empty) set of group by keys in V2/MSQE. + * AggregateOperator is used to aggregate values over a (potentially empty) set of group by keys in V2/MSQE. * Output data will be in the format of [group by key, aggregate result1, ... aggregate resultN] * When the list of aggregation calls is empty, this class is used to calculate distinct result based on group by keys. */ @@ -85,6 +85,7 @@ public class AggregateOperator extends MultiStageOperator { // trimming - related members private final int _groupTrimSize; + @Nullable private final PriorityQueue _priorityQueue; public AggregateOperator(OpChainExecutionContext context, MultiStageOperator input, AggregateNode node) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java index e37798df0888..b3738db6d332 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java @@ -70,16 +70,15 @@ public class MultistageGroupByExecutor { // because they use the zero based integer indexes to store results. private final GroupIdGenerator _groupIdGenerator; - public MultistageGroupByExecutor( - int[] groupKeyIds, - AggregationFunction[] aggFunctions, - int[] filterArgIds, - int maxFilterArgId, - AggType aggType, - boolean leafReturnFinalResult, - DataSchema resultSchema, - Map opChainMetadata, - @Nullable PlanNode.NodeHint nodeHint) { + public MultistageGroupByExecutor(int[] groupKeyIds, + AggregationFunction[] aggFunctions, + int[] filterArgIds, + int maxFilterArgId, + AggType aggType, + boolean leafReturnFinalResult, + DataSchema resultSchema, + Map opChainMetadata, + @Nullable PlanNode.NodeHint nodeHint) { _groupKeyIds = groupKeyIds; _aggFunctions = aggFunctions; _filterArgIds = filterArgIds; From c06caef6773965d723218930aabf577e6117d9f7 Mon Sep 17 00:00:00 2001 From: Bolek Ziobrowski <26925920+bziobrowski@users.noreply.github.com> Date: Thu, 9 Jan 2025 12:53:53 +0100 Subject: [PATCH 5/6] Formatting change. --- .../operator/MultistageGroupByExecutor.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java index b3738db6d332..e37798df0888 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultistageGroupByExecutor.java @@ -70,15 +70,16 @@ public class MultistageGroupByExecutor { // because they use the zero based integer indexes to store results. private final GroupIdGenerator _groupIdGenerator; - public MultistageGroupByExecutor(int[] groupKeyIds, - AggregationFunction[] aggFunctions, - int[] filterArgIds, - int maxFilterArgId, - AggType aggType, - boolean leafReturnFinalResult, - DataSchema resultSchema, - Map opChainMetadata, - @Nullable PlanNode.NodeHint nodeHint) { + public MultistageGroupByExecutor( + int[] groupKeyIds, + AggregationFunction[] aggFunctions, + int[] filterArgIds, + int maxFilterArgId, + AggType aggType, + boolean leafReturnFinalResult, + DataSchema resultSchema, + Map opChainMetadata, + @Nullable PlanNode.NodeHint nodeHint) { _groupKeyIds = groupKeyIds; _aggFunctions = aggFunctions; _filterArgIds = filterArgIds; From 94f20d680409bf5f89fefdbceaa1a506ad86d23c Mon Sep 17 00:00:00 2001 From: Bolek Ziobrowski <26925920+bziobrowski@users.noreply.github.com> Date: Fri, 10 Jan 2025 13:58:02 +0100 Subject: [PATCH 6/6] Reverted reformat. --- .../pinot/core/query/reduce/BrokerReduceService.java | 3 +-- .../pinot/core/query/reduce/GroupByDataTableReducer.java | 9 +++------ 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index 3bf174e7c648..d10e0811ede9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -59,8 +59,7 @@ public BrokerReduceService(PinotConfiguration config) { } public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, BrokerRequest serverBrokerRequest, - Map dataTableMap, - long reduceTimeOutMs, BrokerMetrics brokerMetrics) { + Map dataTableMap, long reduceTimeOutMs, BrokerMetrics brokerMetrics) { if (dataTableMap.isEmpty()) { // Empty response. return BrokerResponseNative.empty(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java index d06778538a03..c53be31ed518 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java @@ -141,12 +141,9 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, * @param brokerMetrics broker metrics (meters) * @throws TimeoutException If unable complete within timeout. */ - private void reduceResult(BrokerResponseNative brokerResponseNative, - DataSchema dataSchema, - Collection dataTables, - DataTableReducerContext reducerContext, - String rawTableName, - BrokerMetrics brokerMetrics) + private void reduceResult(BrokerResponseNative brokerResponseNative, DataSchema dataSchema, + Collection dataTables, DataTableReducerContext reducerContext, String rawTableName, + BrokerMetrics brokerMetrics) throws TimeoutException { // NOTE: This step will modify the data schema and also return final aggregate results. IndexedTable indexedTable = getIndexedTable(dataSchema, dataTables, reducerContext);