-
Notifications
You must be signed in to change notification settings - Fork 5.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Implement constraints based optimization rules
Implements iterative optimizers that look to exploit logical properties propagated as per the previous commit. Note that if the session variable exploit_constraints=false (the default now) no attempt is made to compute logical properties and the optimization rules commited here will not fire.
- Loading branch information
Showing
25 changed files
with
1,505 additions
and
49 deletions.
There are no files selected for viewing
This file contains 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 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
80 changes: 80 additions & 0 deletions
80
...java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantAggregateDistinct.java
This file contains 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,80 @@ | ||
/* | ||
* 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 com.facebook.presto.sql.planner.iterative.rule; | ||
|
||
import com.facebook.presto.matching.Captures; | ||
import com.facebook.presto.matching.Pattern; | ||
import com.facebook.presto.spi.plan.AggregationNode; | ||
import com.facebook.presto.spi.relation.VariableReferenceExpression; | ||
import com.facebook.presto.sql.planner.iterative.GroupReference; | ||
import com.facebook.presto.sql.planner.iterative.Rule; | ||
|
||
import java.util.stream.Collectors; | ||
import java.util.stream.Stream; | ||
|
||
import static com.facebook.presto.spi.plan.AggregationNode.Aggregation.removeDistinct; | ||
import static com.facebook.presto.sql.planner.plan.Patterns.aggregation; | ||
|
||
/** | ||
* Removes distinct from aggregates where the combination of aggregate columns and grouping variables contain a unique key. | ||
* Ultimately this optimization needs to happen before the mark distinct optimization occurs. | ||
* This will require moving the operations that transform away original expressions earlier in the sequence | ||
* as logical property computation is designed to sit behind that transformation. For now this rule | ||
* can be tested by disabling the mark distinct rule. | ||
*/ | ||
public class RemoveRedundantAggregateDistinct | ||
implements Rule<AggregationNode> | ||
{ | ||
private static final Pattern<AggregationNode> PATTERN = aggregation() | ||
.matching(RemoveRedundantAggregateDistinct::aggregateDistinctOfUniqueKey); | ||
|
||
private static boolean aggregateDistinctOfUniqueKey(AggregationNode node) | ||
{ | ||
return !node.getAggregations().isEmpty() && | ||
((GroupReference) node.getSource()).getLogicalProperties().isPresent() && | ||
node.getAggregations().values().stream() | ||
.filter(AggregationNode.Aggregation::isDistinct) | ||
.anyMatch(a -> ((GroupReference) node.getSource()).getLogicalProperties().get().isDistinct( | ||
Stream.concat(node.getGroupingKeys().stream().map(VariableReferenceExpression.class::cast), | ||
a.getArguments().stream().map(VariableReferenceExpression.class::cast)).collect(Collectors.toSet()))); | ||
} | ||
|
||
@Override | ||
public Pattern<AggregationNode> getPattern() | ||
{ | ||
return PATTERN; | ||
} | ||
|
||
@Override | ||
public Result apply(AggregationNode node, Captures captures, Context context) | ||
{ | ||
//create new AggregateNode same as original but with distinct turned off for | ||
//any aggregate function whose argument variables + grouping variables form a unique key | ||
return Result.ofPlanNode(new AggregationNode( | ||
node.getSourceLocation(), | ||
context.getIdAllocator().getNextId(), | ||
node.getSource(), | ||
node.getAggregations().entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> | ||
(e.getValue().isDistinct() && | ||
((GroupReference) node.getSource()).getLogicalProperties().get().isDistinct( | ||
Stream.concat(node.getGroupingKeys().stream().map(VariableReferenceExpression.class::cast), | ||
(e.getValue()).getArguments().stream().map(VariableReferenceExpression.class::cast)).collect(Collectors.toSet()))) ? | ||
removeDistinct(e.getValue()) : (e.getValue()))), | ||
node.getGroupingSets(), | ||
node.getPreGroupedVariables(), | ||
node.getStep(), | ||
node.getHashVariable(), | ||
node.getGroupIdVariable())); | ||
} | ||
} |
55 changes: 55 additions & 0 deletions
55
...src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinct.java
This file contains 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,55 @@ | ||
/* | ||
* 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 com.facebook.presto.sql.planner.iterative.rule; | ||
|
||
import com.facebook.presto.matching.Captures; | ||
import com.facebook.presto.matching.Pattern; | ||
import com.facebook.presto.spi.plan.AggregationNode; | ||
import com.facebook.presto.sql.planner.iterative.GroupReference; | ||
import com.facebook.presto.sql.planner.iterative.Rule; | ||
|
||
import java.util.stream.Collectors; | ||
|
||
import static com.facebook.presto.sql.planner.plan.Patterns.aggregation; | ||
|
||
/** | ||
* Removes distinct operations where the grouping variables contain a unique key. | ||
*/ | ||
public class RemoveRedundantDistinct | ||
implements Rule<AggregationNode> | ||
{ | ||
private static final Pattern<AggregationNode> PATTERN = aggregation() | ||
.matching(AggregationNode::isDistinct) | ||
.matching(RemoveRedundantDistinct::distinctOfUniqueKey); | ||
|
||
private static boolean distinctOfUniqueKey(AggregationNode node) | ||
{ | ||
return node.hasNonEmptyGroupingSet() && | ||
node.getAggregations().isEmpty() && | ||
((GroupReference) node.getSource()).getLogicalProperties().isPresent() && | ||
((GroupReference) node.getSource()).getLogicalProperties().get().isDistinct(node.getGroupingKeys().stream().collect(Collectors.toSet())); | ||
} | ||
|
||
@Override | ||
public Pattern<AggregationNode> getPattern() | ||
{ | ||
return PATTERN; | ||
} | ||
|
||
@Override | ||
public Result apply(AggregationNode node, Captures captures, Context context) | ||
{ | ||
return Result.ofPlanNode(node.getSource()); | ||
} | ||
} |
48 changes: 48 additions & 0 deletions
48
...ain/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantDistinctLimit.java
This file contains 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,48 @@ | ||
/* | ||
* 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 com.facebook.presto.sql.planner.iterative.rule; | ||
|
||
import com.facebook.presto.matching.Captures; | ||
import com.facebook.presto.matching.Pattern; | ||
import com.facebook.presto.spi.plan.DistinctLimitNode; | ||
import com.facebook.presto.sql.planner.iterative.GroupReference; | ||
import com.facebook.presto.sql.planner.iterative.Rule; | ||
|
||
import static com.google.common.base.Preconditions.checkArgument; | ||
|
||
public class RemoveRedundantDistinctLimit | ||
implements Rule<DistinctLimitNode> | ||
{ | ||
private static final Pattern<DistinctLimitNode> PATTERN = Pattern.typeOf(DistinctLimitNode.class) | ||
.matching(RemoveRedundantDistinctLimit::singleRowInput); | ||
|
||
private static boolean singleRowInput(DistinctLimitNode node) | ||
{ | ||
return (((GroupReference) node.getSource()).getLogicalProperties().isPresent() && | ||
((GroupReference) node.getSource()).getLogicalProperties().get().isAtMostSingleRow()); | ||
} | ||
|
||
@Override | ||
public Pattern<DistinctLimitNode> getPattern() | ||
{ | ||
return PATTERN; | ||
} | ||
|
||
@Override | ||
public Result apply(DistinctLimitNode node, Captures captures, Context context) | ||
{ | ||
checkArgument(!node.getHashVariable().isPresent(), "HashSymbol should be empty"); | ||
return Result.ofPlanNode(node.getSource()); | ||
} | ||
} |
51 changes: 51 additions & 0 deletions
51
...in/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantLimit.java
This file contains 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,51 @@ | ||
/* | ||
* 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 com.facebook.presto.sql.planner.iterative.rule; | ||
|
||
import com.facebook.presto.matching.Captures; | ||
import com.facebook.presto.matching.Pattern; | ||
import com.facebook.presto.spi.plan.LimitNode; | ||
import com.facebook.presto.sql.planner.iterative.GroupReference; | ||
import com.facebook.presto.sql.planner.iterative.Rule; | ||
|
||
import static com.facebook.presto.sql.planner.plan.Patterns.limit; | ||
|
||
/** | ||
* Remove Limit node when the subplan is guaranteed to produce fewer rows than the limit. | ||
*/ | ||
public class RemoveRedundantLimit | ||
implements Rule<LimitNode> | ||
{ | ||
// Applies to both LimitNode with ties and LimitNode without ties. | ||
private static final Pattern<LimitNode> PATTERN = limit() | ||
.matching(RemoveRedundantLimit::isAtMost); | ||
|
||
@Override | ||
public Pattern<LimitNode> getPattern() | ||
{ | ||
return PATTERN; | ||
} | ||
|
||
private static boolean isAtMost(LimitNode node) | ||
{ | ||
return ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && | ||
((GroupReference) node.getSource()).getLogicalProperties().get().isAtMost(node.getCount()); | ||
} | ||
|
||
@Override | ||
public Result apply(LimitNode limit, Captures captures, Context context) | ||
{ | ||
return Result.ofPlanNode(limit.getSource()); | ||
} | ||
} |
50 changes: 50 additions & 0 deletions
50
...ain/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantSort.java
This file contains 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,50 @@ | ||
/* | ||
* 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 com.facebook.presto.sql.planner.iterative.rule; | ||
|
||
import com.facebook.presto.matching.Captures; | ||
import com.facebook.presto.matching.Pattern; | ||
import com.facebook.presto.sql.planner.iterative.GroupReference; | ||
import com.facebook.presto.sql.planner.iterative.Rule; | ||
import com.facebook.presto.sql.planner.plan.SortNode; | ||
|
||
import static com.facebook.presto.sql.planner.plan.Patterns.sort; | ||
|
||
/** | ||
* Removes sort operations where the input is provably at most one row. | ||
*/ | ||
public class RemoveRedundantSort | ||
implements Rule<SortNode> | ||
{ | ||
private static final Pattern<SortNode> PATTERN = sort() | ||
.matching(RemoveRedundantSort::singleRowInput); | ||
|
||
private static boolean singleRowInput(SortNode node) | ||
{ | ||
return ((GroupReference) node.getSource()).getLogicalProperties().isPresent() && | ||
((GroupReference) node.getSource()).getLogicalProperties().get().isAtMostSingleRow(); | ||
} | ||
|
||
@Override | ||
public Pattern<SortNode> getPattern() | ||
{ | ||
return PATTERN; | ||
} | ||
|
||
@Override | ||
public Result apply(SortNode node, Captures captures, Context context) | ||
{ | ||
return Result.ofPlanNode(node.getSource()); | ||
} | ||
} |
51 changes: 51 additions & 0 deletions
51
...ain/src/main/java/com/facebook/presto/sql/planner/iterative/rule/RemoveRedundantTopN.java
This file contains 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,51 @@ | ||
/* | ||
* 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 com.facebook.presto.sql.planner.iterative.rule; | ||
|
||
import com.facebook.presto.matching.Captures; | ||
import com.facebook.presto.matching.Pattern; | ||
import com.facebook.presto.spi.plan.TopNNode; | ||
import com.facebook.presto.sql.planner.iterative.GroupReference; | ||
import com.facebook.presto.sql.planner.iterative.Rule; | ||
|
||
import static com.facebook.presto.sql.planner.plan.Patterns.topN; | ||
|
||
/** | ||
* Removes top N operations where the input is provably at most one row. | ||
*/ | ||
public class RemoveRedundantTopN | ||
implements Rule<TopNNode> | ||
{ | ||
private static final Pattern<TopNNode> PATTERN = topN() | ||
.matching(RemoveRedundantTopN::singleRowInput); | ||
|
||
private static boolean singleRowInput(TopNNode node) | ||
{ | ||
return (node.getStep() == TopNNode.Step.SINGLE && | ||
((GroupReference) node.getSource()).getLogicalProperties().isPresent() && | ||
((GroupReference) node.getSource()).getLogicalProperties().get().isAtMostSingleRow()); | ||
} | ||
|
||
@Override | ||
public Pattern<TopNNode> getPattern() | ||
{ | ||
return PATTERN; | ||
} | ||
|
||
@Override | ||
public Result apply(TopNNode node, Captures captures, Context context) | ||
{ | ||
return Result.ofPlanNode(node.getSource()); | ||
} | ||
} |
Oops, something went wrong.