-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-20895] [flink-table-planner] support local aggregate push down in table planner #17344
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
Closed
Closed
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
95a1865
[FLINK-20895] support local aggregate push down in blink planner
sebastianliu f468eca
[FLINK-20895] make changes according to review comments
iyupeng 4ceb389
[FLINK-20895] code and comment optimizations
iyupeng 1dbeace
[FLINK-20895] fix test errors in CI pipelines
iyupeng 289a257
[FLINK-20895] add more rules and tests for local agg push down
iyupeng 0b0584f
[FLINK-20895] optimize code and add test cases for auxGrouping
iyupeng aec0bc1
[FLINK-20895] enable push down local aggregate for existing unit tests
iyupeng File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
207 changes: 207 additions & 0 deletions
207
...main/java/org/apache/flink/table/planner/plan/abilities/source/AggregatePushDownSpec.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,207 @@ | ||
/* | ||
* 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.flink.table.planner.plan.abilities.source; | ||
|
||
import org.apache.flink.table.api.TableException; | ||
import org.apache.flink.table.connector.source.DynamicTableSource; | ||
import org.apache.flink.table.connector.source.abilities.SupportsAggregatePushDown; | ||
import org.apache.flink.table.expressions.AggregateExpression; | ||
import org.apache.flink.table.expressions.FieldReferenceExpression; | ||
import org.apache.flink.table.planner.functions.aggfunctions.AvgAggFunction; | ||
import org.apache.flink.table.planner.functions.aggfunctions.CountAggFunction; | ||
import org.apache.flink.table.planner.functions.aggfunctions.Sum0AggFunction; | ||
import org.apache.flink.table.planner.plan.utils.AggregateInfo; | ||
import org.apache.flink.table.planner.plan.utils.AggregateInfoList; | ||
import org.apache.flink.table.planner.plan.utils.AggregateUtil; | ||
import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; | ||
import org.apache.flink.table.types.DataType; | ||
import org.apache.flink.table.types.logical.RowType; | ||
import org.apache.flink.table.types.utils.TypeConversions; | ||
|
||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; | ||
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName; | ||
|
||
import org.apache.calcite.rel.core.AggregateCall; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
|
||
import scala.Tuple2; | ||
|
||
import static org.apache.flink.util.Preconditions.checkArgument; | ||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
/** | ||
* A sub-class of {@link SourceAbilitySpec} that can not only serialize/deserialize the aggregation | ||
* to/from JSON, but also can push the local aggregate into a {@link SupportsAggregatePushDown}. | ||
*/ | ||
@JsonTypeName("AggregatePushDown") | ||
public class AggregatePushDownSpec extends SourceAbilitySpecBase { | ||
|
||
public static final String FIELD_NAME_INPUT_TYPE = "inputType"; | ||
|
||
public static final String FIELD_NAME_GROUPING_SETS = "groupingSets"; | ||
|
||
public static final String FIELD_NAME_AGGREGATE_CALLS = "aggregateCalls"; | ||
|
||
@JsonProperty(FIELD_NAME_INPUT_TYPE) | ||
private final RowType inputType; | ||
|
||
@JsonProperty(FIELD_NAME_GROUPING_SETS) | ||
private final List<int[]> groupingSets; | ||
|
||
@JsonProperty(FIELD_NAME_AGGREGATE_CALLS) | ||
private final List<AggregateCall> aggregateCalls; | ||
|
||
@JsonCreator | ||
public AggregatePushDownSpec( | ||
@JsonProperty(FIELD_NAME_INPUT_TYPE) RowType inputType, | ||
@JsonProperty(FIELD_NAME_GROUPING_SETS) List<int[]> groupingSets, | ||
@JsonProperty(FIELD_NAME_AGGREGATE_CALLS) List<AggregateCall> aggregateCalls, | ||
@JsonProperty(FIELD_NAME_PRODUCED_TYPE) RowType producedType) { | ||
super(producedType); | ||
|
||
this.inputType = inputType; | ||
this.groupingSets = new ArrayList<>(checkNotNull(groupingSets)); | ||
this.aggregateCalls = aggregateCalls; | ||
} | ||
|
||
@Override | ||
public void apply(DynamicTableSource tableSource, SourceAbilityContext context) { | ||
checkArgument(getProducedType().isPresent()); | ||
apply( | ||
inputType, | ||
groupingSets, | ||
aggregateCalls, | ||
getProducedType().get(), | ||
tableSource, | ||
context); | ||
} | ||
|
||
@Override | ||
public String getDigests(SourceAbilityContext context) { | ||
int[] grouping = groupingSets.get(0); | ||
String groupingStr = | ||
Arrays.stream(grouping) | ||
.mapToObj(index -> inputType.getFieldNames().get(index)) | ||
.collect(Collectors.joining(",")); | ||
|
||
List<AggregateExpression> aggregateExpressions = | ||
buildAggregateExpressions(inputType, aggregateCalls); | ||
String aggFunctionsStr = | ||
aggregateExpressions.stream() | ||
.map(AggregateExpression::asSummaryString) | ||
.collect(Collectors.joining(",")); | ||
|
||
return "aggregates=[grouping=[" | ||
+ groupingStr | ||
+ "], aggFunctions=[" | ||
+ aggFunctionsStr | ||
+ "]]"; | ||
} | ||
|
||
public static boolean apply( | ||
RowType inputType, | ||
List<int[]> groupingSets, | ||
List<AggregateCall> aggregateCalls, | ||
RowType producedType, | ||
DynamicTableSource tableSource, | ||
SourceAbilityContext context) { | ||
assert context.isBatchMode() && groupingSets.size() == 1; | ||
|
||
List<AggregateExpression> aggregateExpressions = | ||
buildAggregateExpressions(inputType, aggregateCalls); | ||
|
||
if (tableSource instanceof SupportsAggregatePushDown) { | ||
DataType producedDataType = TypeConversions.fromLogicalToDataType(producedType); | ||
return ((SupportsAggregatePushDown) tableSource) | ||
.applyAggregates(groupingSets, aggregateExpressions, producedDataType); | ||
} else { | ||
throw new TableException( | ||
String.format( | ||
"%s does not support SupportsAggregatePushDown.", | ||
tableSource.getClass().getName())); | ||
} | ||
} | ||
|
||
private static List<AggregateExpression> buildAggregateExpressions( | ||
RowType inputType, List<AggregateCall> aggregateCalls) { | ||
AggregateInfoList aggInfoList = | ||
AggregateUtil.transformToBatchAggregateInfoList( | ||
iyupeng marked this conversation as resolved.
Show resolved
Hide resolved
|
||
inputType, JavaScalaConversionUtil.toScala(aggregateCalls), null, null); | ||
if (aggInfoList.aggInfos().length == 0) { | ||
// no agg function need to be pushed down | ||
return Collections.emptyList(); | ||
} | ||
|
||
List<AggregateExpression> aggExpressions = new ArrayList<>(); | ||
for (AggregateInfo aggInfo : aggInfoList.aggInfos()) { | ||
List<FieldReferenceExpression> arguments = new ArrayList<>(1); | ||
for (int argIndex : aggInfo.argIndexes()) { | ||
DataType argType = | ||
TypeConversions.fromLogicalToDataType( | ||
inputType.getFields().get(argIndex).getType()); | ||
FieldReferenceExpression field = | ||
new FieldReferenceExpression( | ||
inputType.getFieldNames().get(argIndex), argType, 0, argIndex); | ||
arguments.add(field); | ||
} | ||
if (aggInfo.function() instanceof AvgAggFunction) { | ||
Tuple2<Sum0AggFunction, CountAggFunction> sum0AndCountFunction = | ||
AggregateUtil.deriveSumAndCountFromAvg((AvgAggFunction) aggInfo.function()); | ||
AggregateExpression sum0Expression = | ||
new AggregateExpression( | ||
sum0AndCountFunction._1(), | ||
arguments, | ||
null, | ||
aggInfo.externalResultType(), | ||
aggInfo.agg().isDistinct(), | ||
aggInfo.agg().isApproximate(), | ||
aggInfo.agg().ignoreNulls()); | ||
aggExpressions.add(sum0Expression); | ||
AggregateExpression countExpression = | ||
new AggregateExpression( | ||
sum0AndCountFunction._2(), | ||
arguments, | ||
null, | ||
aggInfo.externalResultType(), | ||
aggInfo.agg().isDistinct(), | ||
aggInfo.agg().isApproximate(), | ||
aggInfo.agg().ignoreNulls()); | ||
aggExpressions.add(countExpression); | ||
} else { | ||
AggregateExpression aggregateExpression = | ||
new AggregateExpression( | ||
aggInfo.function(), | ||
arguments, | ||
null, | ||
aggInfo.externalResultType(), | ||
aggInfo.agg().isDistinct(), | ||
aggInfo.agg().isApproximate(), | ||
aggInfo.agg().ignoreNulls()); | ||
aggExpressions.add(aggregateExpression); | ||
} | ||
} | ||
return aggExpressions; | ||
} | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
@iyupeng are we sure that the JavaDocs don't need more updates for this PR?
For example, is this description still correct?
Also maybe we should mention the config option that was added in this PR.
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.
Hi @twalthr, the strategy is still
all or nothing
here.applyAggregates
returns a bool value, meaning whether all local aggregates are accepted by underlying data source or not at all.The docs could be improved, like mentioning the new option, etc.
I could push a hotfix for docs.
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.
That would be great. The more information in the interface description, the better for all implementers.
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.
Hi @twalthr I pushed this for doc improvement: #17630