-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-21923][table-planner-blink] Fix SplitAggregateRule not support both filter, count/sum, avg #15341
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-21923][table-planner-blink] Fix SplitAggregateRule not support both filter, count/sum, avg #15341
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery | |
| import org.apache.flink.table.planner.plan.nodes.FlinkRelNode | ||
| import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate | ||
| import org.apache.flink.table.planner.plan.utils.AggregateUtil.doAllAggSupportSplit | ||
| import org.apache.flink.table.planner.plan.utils.{ExpandUtil, WindowUtil} | ||
| import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ExpandUtil, WindowUtil} | ||
|
|
||
| import org.apache.calcite.plan.RelOptRule.{any, operand} | ||
| import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} | ||
|
|
@@ -138,9 +138,11 @@ class SplitAggregateRule extends RelOptRule( | |
| val windowProps = fmq.getRelWindowProperties(agg.getInput) | ||
| val isWindowAgg = WindowUtil.groupingContainsWindowStartEnd(agg.getGroupSet, windowProps) | ||
| val isProctimeWindowAgg = isWindowAgg && !windowProps.isRowtime | ||
| // TableAggregate is not supported. see also FLINK-21923. | ||
| val isTableAgg = AggregateUtil.isTableAggregate(agg.getAggCallList) | ||
|
|
||
| agg.partialFinalType == PartialFinalType.NONE && agg.containsDistinctCall() && | ||
| splitDistinctAggEnabled && isAllAggSplittable && !isProctimeWindowAgg | ||
| splitDistinctAggEnabled && isAllAggSplittable && !isProctimeWindowAgg && !isTableAgg | ||
| } | ||
|
|
||
| override def onMatch(call: RelOptRuleCall): Unit = { | ||
|
|
@@ -280,11 +282,16 @@ class SplitAggregateRule extends RelOptRule( | |
| } | ||
|
|
||
| // STEP 2.3: construct partial aggregates | ||
| relBuilder.aggregate( | ||
| relBuilder.groupKey(fullGroupSet, ImmutableList.of[ImmutableBitSet](fullGroupSet)), | ||
| // Create aggregate node directly to avoid ClassCastException, | ||
| // Please see https://issues.apache.org/jira/browse/FLINK-21923 for more details. | ||
| // TODO reuse aggregate function, see FLINK-22412 | ||
| val partialAggregate = FlinkLogicalAggregate.create( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The current approach does not consider the optimization of aggregate function reuse
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The implementation of the previous version used calcite's function reuse optimization.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For a quick fix, I suggest we keep the current approach and create a new issue to tract the optimization (please also add some comments here). you can take the optimization issue if you like
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for your reminder! I has created a new issue FLINK-22412
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add a TODO here (such as
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok,I will deal with it as soon as possible
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have added TODO, please take a look again, thanks |
||
| relBuilder.build(), | ||
| fullGroupSet, | ||
| ImmutableList.of[ImmutableBitSet](fullGroupSet), | ||
| newPartialAggCalls) | ||
| relBuilder.peek().asInstanceOf[FlinkLogicalAggregate] | ||
| .setPartialFinalType(PartialFinalType.PARTIAL) | ||
| partialAggregate.setPartialFinalType(PartialFinalType.PARTIAL) | ||
| relBuilder.push(partialAggregate) | ||
|
|
||
| // STEP 3: construct final aggregates | ||
| val finalAggInputOffset = fullGroupSet.cardinality | ||
|
|
@@ -306,13 +313,16 @@ class SplitAggregateRule extends RelOptRule( | |
| needMergeFinalAggOutput = true | ||
| } | ||
| } | ||
| relBuilder.aggregate( | ||
| relBuilder.groupKey( | ||
| SplitAggregateRule.remap(fullGroupSet, originalAggregate.getGroupSet), | ||
| SplitAggregateRule.remap(fullGroupSet, Seq(originalAggregate.getGroupSet))), | ||
| // Create aggregate node directly to avoid ClassCastException, | ||
| // Please see https://issues.apache.org/jira/browse/FLINK-21923 for more details. | ||
| // TODO reuse aggregate function, see FLINK-22412 | ||
| val finalAggregate = FlinkLogicalAggregate.create( | ||
| relBuilder.build(), | ||
| SplitAggregateRule.remap(fullGroupSet, originalAggregate.getGroupSet), | ||
| SplitAggregateRule.remap(fullGroupSet, Seq(originalAggregate.getGroupSet)), | ||
| finalAggCalls) | ||
| val finalAggregate = relBuilder.peek().asInstanceOf[FlinkLogicalAggregate] | ||
| finalAggregate.setPartialFinalType(PartialFinalType.FINAL) | ||
| relBuilder.push(finalAggregate) | ||
|
|
||
| // STEP 4: convert final aggregation output to the original aggregation output. | ||
| // For example, aggregate function AVG is transformed to SUM0 and COUNT, so the output of | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I do not find any explanation why TableAggregate is not supported in FLINK-21923
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I left a message here FLINK-21923.
If it is TableAggregate, ClassCastException will definitely occur