-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Limit number of nodes that execute writing stages #15987
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
radek-kondziolka
wants to merge
3
commits into
trinodb:master
from
radek-kondziolka:rk/add_upper_limit_on_writer_scaling_backup
Closed
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
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
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
136 changes: 136 additions & 0 deletions
136
...trino-main/src/main/java/io/trino/sql/planner/optimizations/LimitMaxWriterNodesCount.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,136 @@ | ||
| /* | ||
| * Licensed 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 io.trino.sql.planner.optimizations; | ||
|
|
||
| import com.google.common.collect.ImmutableList; | ||
| import io.trino.Session; | ||
| import io.trino.cost.TableStatsProvider; | ||
| import io.trino.execution.warnings.WarningCollector; | ||
| import io.trino.operator.RetryPolicy; | ||
| import io.trino.sql.planner.PartitioningHandle; | ||
| import io.trino.sql.planner.PlanNodeIdAllocator; | ||
| import io.trino.sql.planner.SymbolAllocator; | ||
| import io.trino.sql.planner.SystemPartitioningHandle; | ||
| import io.trino.sql.planner.TypeProvider; | ||
| import io.trino.sql.planner.plan.ExchangeNode; | ||
| import io.trino.sql.planner.plan.PlanNode; | ||
| import io.trino.sql.planner.plan.SimplePlanRewriter; | ||
| import io.trino.sql.planner.plan.TableWriterNode; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| import static com.google.common.collect.ImmutableList.toImmutableList; | ||
| import static io.trino.SystemSessionProperties.MAX_HASH_PARTITION_COUNT; | ||
| import static io.trino.SystemSessionProperties.MAX_WRITER_NODES_COUNT; | ||
| import static io.trino.SystemSessionProperties.getRetryPolicy; | ||
| import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; | ||
| import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_HASH_DISTRIBUTION; | ||
| import static io.trino.sql.planner.SystemPartitioningHandle.SCALED_WRITER_HASH_DISTRIBUTION; | ||
| import static io.trino.sql.planner.plan.ExchangeNode.Scope.REMOTE; | ||
| import static io.trino.sql.planner.plan.SimplePlanRewriter.rewriteWith; | ||
| import static java.util.Objects.requireNonNull; | ||
|
|
||
| public class LimitMaxWriterNodesCount | ||
| implements PlanOptimizer | ||
| { | ||
| private static final List<Class<? extends PlanNode>> SUPPORTED_NODES = ImmutableList.of(TableWriterNode.class); | ||
| private static final List<PartitioningHandle> SUPPORTED_PARTITIONING_MODES = ImmutableList.of( | ||
| FIXED_HASH_DISTRIBUTION, SCALED_WRITER_HASH_DISTRIBUTION, FIXED_ARBITRARY_DISTRIBUTION); | ||
|
|
||
| @Override | ||
| public PlanNode optimize( | ||
| PlanNode plan, | ||
| Session session, | ||
| TypeProvider types, | ||
| SymbolAllocator symbolAllocator, | ||
| PlanNodeIdAllocator idAllocator, | ||
| WarningCollector warningCollector, | ||
| TableStatsProvider tableStatsProvider) | ||
| { | ||
| requireNonNull(plan, "plan is null"); | ||
| requireNonNull(session, "session is null"); | ||
|
|
||
| // Skip for plans where there is not writing stages. Additionally, skip for FTE mode since we | ||
| // are not using estimated partitionCount in FTE scheduler. | ||
|
|
||
| if (!PlanNodeSearcher.searchFrom(plan).whereIsInstanceOfAny(SUPPORTED_NODES).matches() || getRetryPolicy(session) == RetryPolicy.TASK) { | ||
| return plan; | ||
| } | ||
|
|
||
| // if TableWriter's source is not an exchange or partitioning is not supported does not fire that rule | ||
| List<TableWriterNode> allTableWriters = PlanNodeSearcher | ||
| .searchFrom(plan) | ||
| .where(TableWriterNode.class::isInstance) | ||
| .findAll(); | ||
|
|
||
| boolean isPartitioningWritingModeSupported = allTableWriters | ||
| .stream() | ||
| .allMatch(it -> it.getSource() instanceof ExchangeNode exchangeNode && SUPPORTED_PARTITIONING_MODES.contains(exchangeNode.getPartitioningScheme().getPartitioning().getHandle())); | ||
|
|
||
| if (!isPartitioningWritingModeSupported) { | ||
| return plan; | ||
| } | ||
|
|
||
| // if there is not-system partitioning does not file that rule | ||
| boolean isAllPartitioningSystemPartitioning = PlanNodeSearcher | ||
| .searchFrom(plan) | ||
| .where(ExchangeNode.class::isInstance) | ||
| .findAll() | ||
| .stream() | ||
| .map(ExchangeNode.class::cast) | ||
| .map(node -> node.getPartitioningScheme().getPartitioning().getHandle().getConnectorHandle()) | ||
| .allMatch(SystemPartitioningHandle.class::isInstance); | ||
|
|
||
| if (!isAllPartitioningSystemPartitioning) { | ||
| return plan; | ||
| } | ||
|
|
||
| int maxWritersNodesCount = Math.min( | ||
| session.getSystemProperty(MAX_WRITER_NODES_COUNT, Integer.class), | ||
| session.getSystemProperty(MAX_HASH_PARTITION_COUNT, Integer.class)); | ||
| return rewriteWith(new Rewriter(maxWritersNodesCount), plan); | ||
| } | ||
|
|
||
| private static class Rewriter | ||
| extends SimplePlanRewriter<Void> | ||
| { | ||
| private final int maxWriterNodesCount; | ||
|
|
||
| private Rewriter(int maxWriterNodesCount) | ||
| { | ||
| this.maxWriterNodesCount = maxWriterNodesCount; | ||
| } | ||
|
|
||
| @Override | ||
| public PlanNode visitExchange(ExchangeNode node, RewriteContext<Void> context) | ||
| { | ||
| if (node.getScope() != REMOTE) { | ||
| return node; | ||
| } | ||
|
|
||
| List<PlanNode> sources = node.getSources().stream() | ||
| .map(context::rewrite) | ||
| .collect(toImmutableList()); | ||
|
|
||
| return new ExchangeNode( | ||
| node.getId(), | ||
| node.getType(), | ||
| node.getScope(), | ||
| node.getPartitioningScheme().withPartitionCount(maxWriterNodesCount), | ||
| sources, | ||
| node.getInputs(), | ||
| node.getOrderingScheme()); | ||
| } | ||
| } | ||
| } | ||
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.
Instead of having a separate rule, you can do this directly inside
AddExchange#visitTableWriterThere 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.
Yes, it could. I do it here because:
PartitioningScheme-ApplyPreferredTableExecutePartitioning. This rule is very simple, just basing on configuration toggle and could be part ofAddExchangesas well.This is why I decided to wrap it within another rule.
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.
TBH, I don't think we need a separate rule for this since we are just applying the configuration. In
ApplyPreferredTableExecutePartitioningwe are taking a decision based on estimates to use preferred partitioning or not. So, Its kinda makes sense.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.
And in future if we ever make it adaptive based on stats we can always add a new rule and remove from
AddExchanges. It shouldn't be a big change.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.
WDYT? @sopel39
Uh oh!
There was an error while loading. Please reload this page.
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 think the easier approach is preferred. If it's just few lines of code, then
AddExchange#visitTableWriteris good enough.Be sure to have proper testing coverage. With
AddExchangeI think you have to haveBasePlanTestkind of testsThere 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.
@gaurav8297 , the rule
LimitMaxWriterNodesCountis now much complicated. Especially, we skip the rule in some cases that is not skipped inAddExchangerule. I do not think that we should merge them. It is complicated it seems to be at the beginning.