From 13c22112dd0e4e79c140674693a83dc72f22f847 Mon Sep 17 00:00:00 2001 From: David Simmen Date: Mon, 6 Jun 2022 17:39:39 -0400 Subject: [PATCH] Compute and propagate logical properties Logical properties are initially derived from constraints defined for base tables and from properties of values nodes.These logical properties hold for the result table produced by a plan node. These base logical properties are then propagated through various query operations including filters, projects, joins, and aggregations. Logical properties are only computed by iterative planners that pass a logical property provider as input. See the design doc linked from issue 16413 for futher details. Such optimizers will be introduced by next commit; however, there are test cases in this commit that trigger logical property propgation. Note that if the session variable exploit_constraints=false (the default now) no attempt is made to compute logical properties and hence optimization rules that seek them out will simply fail to fire. --- .../presto/sql/planner/PlanOptimizers.java | 4 + .../sql/planner/iterative/GroupReference.java | 10 +- .../planner/iterative/IterativeOptimizer.java | 24 +- .../presto/sql/planner/iterative/Memo.java | 50 +- .../properties/EquivalenceClassProperty.java | 311 +++ .../sql/planner/iterative/properties/Key.java | 151 ++ .../iterative/properties/KeyProperty.java | 216 ++ .../properties/LogicalPropertiesImpl.java | 644 +++++ .../LogicalPropertiesProviderImpl.java | 345 +++ .../iterative/properties/MaxCardProperty.java | 137 + .../sql/planner/plan/AssignUniqueId.java | 9 + .../presto/sql/planner/plan/JoinNode.java | 9 + .../presto/sql/planner/plan/SemiJoinNode.java | 9 + .../presto/sql/planner/plan/SortNode.java | 9 + .../sql/relational/FunctionResolution.java | 5 + .../rule/TestLogicalPropertyPropagation.java | 2253 +++++++++++++++++ .../iterative/rule/test/BaseRuleTest.java | 4 +- .../iterative/rule/test/PlanBuilder.java | 37 +- .../iterative/rule/test/RuleAssert.java | 52 +- .../iterative/rule/test/RuleTester.java | 25 +- .../presto/spi/plan/AggregationNode.java | 17 + .../presto/spi/plan/DistinctLimitNode.java | 7 + .../facebook/presto/spi/plan/FilterNode.java | 8 + .../facebook/presto/spi/plan/LimitNode.java | 7 + .../presto/spi/plan/LogicalProperties.java | 56 + .../spi/plan/LogicalPropertiesProvider.java | 55 + .../facebook/presto/spi/plan/PlanNode.java | 9 + .../facebook/presto/spi/plan/ProjectNode.java | 7 + .../presto/spi/plan/TableScanNode.java | 7 + .../facebook/presto/spi/plan/TopNNode.java | 7 + .../facebook/presto/spi/plan/ValuesNode.java | 7 + 31 files changed, 4466 insertions(+), 25 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.java create mode 100644 presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java index 699f73fb5b2ed..ea5677e1b4f2c 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java @@ -24,6 +24,7 @@ import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.planner.iterative.IterativeOptimizer; import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; import com.facebook.presto.sql.planner.iterative.rule.AddIntermediateAggregations; import com.facebook.presto.sql.planner.iterative.rule.CanonicalizeExpressions; import com.facebook.presto.sql.planner.iterative.rule.CreatePartialTopN; @@ -140,6 +141,7 @@ import com.facebook.presto.sql.planner.optimizations.TransformQuantifiedComparisonApplyToLateralJoin; import com.facebook.presto.sql.planner.optimizations.UnaliasSymbolReferences; import com.facebook.presto.sql.planner.optimizations.WindowFilterPushDown; +import com.facebook.presto.sql.relational.FunctionResolution; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.weakref.jmx.MBeanExporter; @@ -149,6 +151,7 @@ import javax.inject.Inject; import java.util.List; +import java.util.Optional; import java.util.Set; import static com.facebook.presto.sql.planner.ConnectorPlanOptimizerManager.PlanPhase.LOGICAL; @@ -430,6 +433,7 @@ public PlanOptimizers( ruleStats, statsCalculator, estimatedExchangesCostCalculator, + Optional.of(new LogicalPropertiesProviderImpl(new FunctionResolution(metadata.getFunctionAndTypeManager()))), ImmutableSet.of( new RemoveRedundantIdentityProjections(), new PushAggregationThroughOuterJoin(metadata.getFunctionAndTypeManager()))), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java index 4d6a1daa5d92a..7327d04267471 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/GroupReference.java @@ -14,6 +14,7 @@ package com.facebook.presto.sql.planner.iterative; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -29,12 +30,14 @@ public class GroupReference { private final int groupId; private final List outputs; + private final Optional logicalProperties; - public GroupReference(Optional sourceLocation, PlanNodeId id, int groupId, List outputs) + public GroupReference(Optional sourceLocation, PlanNodeId id, int groupId, List outputs, Optional logicalProperties) { super(sourceLocation, id); this.groupId = groupId; this.outputs = ImmutableList.copyOf(outputs); + this.logicalProperties = logicalProperties; } public int getGroupId() @@ -65,4 +68,9 @@ public PlanNode replaceChildren(List newChildren) { throw new UnsupportedOperationException(); } + + public Optional getLogicalProperties() + { + return logicalProperties; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java index d4dd17cdc780d..1b8d0a465cdc5 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/IterativeOptimizer.java @@ -25,6 +25,7 @@ import com.facebook.presto.matching.Matcher; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; import com.facebook.presto.sql.planner.PlanVariableAllocator; @@ -56,13 +57,24 @@ public class IterativeOptimizer private final CostCalculator costCalculator; private final List legacyRules; private final RuleIndex ruleIndex; + private final Optional logicalPropertiesProvider; public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, Set> rules) { - this(stats, statsCalculator, costCalculator, ImmutableList.of(), rules); + this(stats, statsCalculator, costCalculator, ImmutableList.of(), Optional.empty(), rules); + } + + public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, Optional logicalPropertiesProvider, Set> rules) + { + this(stats, statsCalculator, costCalculator, ImmutableList.of(), logicalPropertiesProvider, rules); } public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, List legacyRules, Set> newRules) + { + this(stats, statsCalculator, costCalculator, legacyRules, Optional.empty(), newRules); + } + + public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculator, CostCalculator costCalculator, List legacyRules, Optional logicalPropertiesProvider, Set> newRules) { this.stats = requireNonNull(stats, "stats is null"); this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); @@ -71,6 +83,7 @@ public IterativeOptimizer(RuleStatsRecorder stats, StatsCalculator statsCalculat this.ruleIndex = RuleIndex.builder() .register(newRules) .build(); + this.logicalPropertiesProvider = requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider is null"); stats.registerAll(newRules); } @@ -87,7 +100,14 @@ public PlanNode optimize(PlanNode plan, Session session, TypeProvider types, Pla return plan; } - Memo memo = new Memo(idAllocator, plan); + Memo memo; + if (SystemSessionProperties.isExploitConstraints(session)) { + memo = new Memo(idAllocator, plan, logicalPropertiesProvider); + } + else { + memo = new Memo(idAllocator, plan, Optional.empty()); + } + Lookup lookup = Lookup.from(planNode -> Stream.of(memo.resolve(planNode))); Matcher matcher = new PlanNodeMatcher(lookup); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java index 679e0c497fde2..65a5bfbfdfffe 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/Memo.java @@ -15,6 +15,8 @@ import com.facebook.presto.cost.PlanCostEstimate; import com.facebook.presto.cost.PlanNodeStatsEstimate; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; import com.google.common.collect.HashMultiset; @@ -69,12 +71,19 @@ public class Memo private final int rootGroup; private final Map groups = new HashMap<>(); + private final Optional logicalPropertiesProvider; private int nextGroupId = ROOT_GROUP_REF + 1; public Memo(PlanNodeIdAllocator idAllocator, PlanNode plan) + { + this(idAllocator, plan, Optional.empty()); + } + + public Memo(PlanNodeIdAllocator idAllocator, PlanNode plan, Optional logicalPropertiesProvider) { this.idAllocator = idAllocator; + this.logicalPropertiesProvider = logicalPropertiesProvider; rootGroup = insertRecursive(plan); groups.get(rootGroup).incomingReferences.add(ROOT_GROUP_REF); } @@ -90,6 +99,12 @@ private Group getGroup(int group) return groups.get(group); } + public Optional getLogicalProperties(int group) + { + checkArgument(groups.containsKey(group), "Invalid group: %s", group); + return groups.get(group).logicalProperties; + } + public PlanNode getNode(int group) { return getGroup(group).membership; @@ -129,6 +144,14 @@ public PlanNode replace(int group, PlanNode node, String reason) incrementReferenceCounts(node, group); getGroup(group).membership = node; + + if (logicalPropertiesProvider.isPresent()) { + // for now, we replace existing group logical properties with those computed for the new node + // as we cannot ensure equivalence for all plans in a group until we support functional dependencies + // once we can ensure equivalence we can simply reuse the previously computed properties for all plans in the group + LogicalProperties newLogicalProperties = node.computeLogicalProperties(logicalPropertiesProvider.get()); + getGroup(group).logicalProperties = Optional.of(newLogicalProperties); + } decrementReferenceCounts(old, group); evictStatisticsAndCost(group); @@ -212,11 +235,15 @@ private PlanNode insertChildrenAndRewrite(PlanNode node) { return node.replaceChildren( node.getSources().stream() - .map(child -> new GroupReference( - node.getSourceLocation(), - idAllocator.getNextId(), - insertRecursive(child), - child.getOutputVariables())) + .map(child -> { + int childId = insertRecursive(child); + return new GroupReference( + node.getSourceLocation(), + idAllocator.getNextId(), + childId, + child.getOutputVariables(), + groups.get(childId).logicalProperties); + }) .collect(Collectors.toList())); } @@ -229,7 +256,7 @@ private int insertRecursive(PlanNode node) int group = nextGroupId(); PlanNode rewritten = insertChildrenAndRewrite(node); - groups.put(group, Group.withMember(rewritten)); + groups.put(group, new Group(rewritten, logicalPropertiesProvider.map(rewritten::computeLogicalProperties))); incrementReferenceCounts(rewritten, group); return group; @@ -247,21 +274,18 @@ public int getGroupCount() private static final class Group { - static Group withMember(PlanNode member) - { - return new Group(member); - } - - private PlanNode membership; private final Multiset incomingReferences = HashMultiset.create(); + private PlanNode membership; + private Optional logicalProperties; @Nullable private PlanNodeStatsEstimate stats; @Nullable private PlanCostEstimate cost; - private Group(PlanNode member) + private Group(PlanNode member, Optional logicalProperties) { this.membership = requireNonNull(member, "member is null"); + this.logicalProperties = logicalProperties; } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java new file mode 100644 index 0000000000000..c683738fb22ed --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/EquivalenceClassProperty.java @@ -0,0 +1,311 @@ +/* + * 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.properties; + +import com.facebook.presto.spi.relation.CallExpression; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.facebook.presto.expressions.LogicalRowExpressions.extractConjuncts; +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +/** + * Represents classes of equivalent variable and constants references that + * hold for a final or intermediate result set produced by a PlanNode. + * Variable and constant references are made equivalent via predicate application. + *

+ * Each equivalence class is represented by a head. The head is carefully chosen + * to be a the member with the strongest binding (i.e. a constant). + *

+ * Note that the equivalence class property does not store trivial equivalence + * classes with only one member. All variable or constant references are + * considered to be in their own virtual equivalence class until combined + * into stored larger classes by predicate application. + */ +public class EquivalenceClassProperty +{ + private final Map equivalenceClassHeads = new HashMap<>(); + private final Map> equivalenceClasses = new HashMap<>(); + private final FunctionResolution functionResolution; + + private boolean updated; + + public EquivalenceClassProperty(FunctionResolution functionResolution) + { + this.functionResolution = functionResolution; + } + + /** + * Determines if one equivalence class property is more general than another. + * An equivalence class property is more general than another if it includes all equivalences of the other. + * + * @param otherEquivalenceClassProperty + * @return True if this equivalenceClassProperty is more general than otherEquivalenceClassProperty or False otherwise. + */ + public boolean isMoreGeneralThan(EquivalenceClassProperty otherEquivalenceClassProperty) + { + requireNonNull(otherEquivalenceClassProperty, "otherEquivalenceClassProperty is null"); + if (equivalenceClasses.isEmpty() && otherEquivalenceClassProperty.equivalenceClasses.isEmpty()) { + return true; + } + if (equivalenceClasses.isEmpty() || otherEquivalenceClassProperty.equivalenceClasses.isEmpty()) { + return false; + } + + ImmutableList> eqClassSets = equivalenceClasses.entrySet() + .stream() + .map(e1 -> new ImmutableSet.Builder() + .add(e1.getKey()) + .addAll(e1.getValue()) + .build()) + .collect(toImmutableList()); + + //every equivalence class of other is a subset of some equivalence class of the first + return otherEquivalenceClassProperty.equivalenceClasses.entrySet() + .stream() + .allMatch(e -> { + final Set otherEqClass = new HashSet<>(); + otherEqClass.add(e.getKey()); + otherEqClass.addAll(e.getValue()); + return eqClassSets.stream().anyMatch(eqClassSet -> eqClassSet.containsAll(otherEqClass)); + }); + } + + /** + * Returns the head of the equivalence class of the provided variable or constant reference. + * + * @param expression + * @return The head of the equivalence class of the provided variable or constant reference. + */ + public RowExpression getEquivalenceClassHead(RowExpression expression) + { + requireNonNull(expression, "expression is null"); + checkArgument((expression instanceof VariableReferenceExpression || expression instanceof ConstantExpression), + "Row expression is of type " + expression.getClass().getSimpleName() + ", must be a VariableReferenceExpression or a ConstantExpression."); + //all variables start out by default in their own virtual singleton class + return equivalenceClassHeads.getOrDefault(expression, expression); + } + + /** + * Returns the equivalence classes members for the given equivalence class head. + *

+ * Note that the provided head could be that of a new equivalence class in which case + * an empty member list is returned. + * + * @param head + * @return The equivalence class members for the given equivalence class head. + */ + public List getEquivalenceClasses(RowExpression head) + { + requireNonNull(head, "head is null"); + checkArgument((head instanceof VariableReferenceExpression || head instanceof ConstantExpression), + "Row expression is of type " + head.getClass().getSimpleName() + ", must be a VariableReferenceExpression or a ConstantExpression."); + + return equivalenceClasses.getOrDefault(head, new ArrayList<>()); + } + + /** + * Updates this equivalence class property with the equivalences of another equivalence class property. + * + * @param equivalenceClassProperty + */ + public void update(EquivalenceClassProperty equivalenceClassProperty) + { + requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + + equivalenceClassProperty.equivalenceClasses.entrySet().forEach(eqClass -> { + final RowExpression head = eqClass.getKey(); + List members = eqClass.getValue(); + members.forEach(member -> updateInternal(head, member)); + }); + } + + /** + * Updates this equivalence class property with "variable reference = variable reference" or + * "variable reference = constant reference" conjuncts applied by the provided predicate. + * Returns true if any equivalence class heads changed, and false otherwise. + * + * @param predicate + * @return + */ + public boolean update(RowExpression predicate) + { + requireNonNull(predicate, "predicate is null"); + updated = false; + //TODO tunnel through CAST functions? + extractConjuncts(predicate).stream() + .filter(CallExpression.class::isInstance) + .map(CallExpression.class::cast) + .filter(e -> isVariableEqualVariableOrConstant(e)) + .forEach(e -> updateInternal(e.getArguments().get(0), e.getArguments().get(1))); + return updated; + } + + private boolean isVariableEqualVariableOrConstant(RowExpression expression) + { + if (expression instanceof CallExpression + && functionResolution.isEqualFunction(((CallExpression) expression).getFunctionHandle()) + && ((CallExpression) expression).getArguments().size() == 2) { + RowExpression e1 = ((CallExpression) expression).getArguments().get(0); + RowExpression e2 = ((CallExpression) expression).getArguments().get(1); + + if ((e1 instanceof VariableReferenceExpression && (e2 instanceof VariableReferenceExpression || e2 instanceof ConstantExpression)) + || ((e2 instanceof VariableReferenceExpression) && (e1 instanceof VariableReferenceExpression || e1 instanceof ConstantExpression))) { + return true; + } + } + return false; + } + + public void update(RowExpression firstExpression, RowExpression secondExpression) + { + updateInternal(firstExpression, secondExpression); + } + + /** + * Updates this equivalence class property with pairs of variable or column references deemed + * equivalent via the application of predicates. + * Side effect is that it sets updated instance variable true if any equivalence class heads changed. + * This can be used to optimize methods that maintain alignment with equivalence classes. + * + * @param firstExpression + * @param secondExpression + */ + private void updateInternal(RowExpression firstExpression, RowExpression secondExpression) + { + RowExpression head1 = getEquivalenceClassHead(firstExpression); + RowExpression head2 = getEquivalenceClassHead(secondExpression); + + //already in same equivalence class, nothing to do + //note that we do not check head1.equal(head2) so that two different variable reference objects + //referencing the same reference are both added to the equivalence class + if (head1 == head2) { + return; + } + + updated = true; + List head1Class = getEquivalenceClasses(head1); + List head2Class = getEquivalenceClasses(head2); + + //pick new head and merge other class into head class + RowExpression newHead = pickNewHead(head1, head2); + if (newHead == head1) { + combineClasses(head1, head1Class, head2, head2Class); + } + else { + combineClasses(head2, head2Class, head1, head1Class); + } + } + + private RowExpression pickNewHead(RowExpression head1, RowExpression head2) + { + //always use constant as the head + if (head1 instanceof ConstantExpression) { + return head1; + } + else if (head2 instanceof ConstantExpression) { + return head2; + } + else { + return head1; + } + } + + //combine an equivalence class with head class + private void combineClasses(RowExpression head, List headClass, RowExpression headOfOtherEqClass, List otherEqClass) + { + //merge other eq class into head class + headClass.addAll(otherEqClass); + headClass.add(headOfOtherEqClass); + //update the head of the other class members + equivalenceClassHeads.put(headOfOtherEqClass, head); + for (RowExpression expression : otherEqClass) { + equivalenceClassHeads.put(expression, head); + } + equivalenceClasses.putIfAbsent(head, headClass); + equivalenceClasses.remove(headOfOtherEqClass); + } + + /** + * Returns a projected version of this equivalence class property. + * Variables in each class are mapped to output variables in the context beyond the project operation. + * It is possible that this operation projects all members of a particular class. + * + * @param inverseVariableMappings + * @return A projected version of this equivalence class property. + */ + public EquivalenceClassProperty project(Map inverseVariableMappings) + { + EquivalenceClassProperty projectedEquivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + for (Map.Entry> entry : equivalenceClasses.entrySet()) { + //first project the members of the current class + List projectedMembers = new ArrayList<>(); + for (RowExpression member : entry.getValue()) { + if (inverseVariableMappings.containsKey(member)) { + RowExpression projectedMember = inverseVariableMappings.get(member); + if (!projectedMembers.contains(projectedMember)) { + projectedMembers.add(projectedMember); + } + } + } + //boundary cases.... + // head projects but no members project -> trivial class, do not store + // head does not project and one member projects -> trivial class, do not store + // head does not project and more than one member projects -> pick a new head and update equivalence class + if (!projectedMembers.isEmpty()) { + RowExpression currentHead = entry.getKey(); + RowExpression projectedHead = currentHead; + if ((currentHead instanceof VariableReferenceExpression)) { + if (inverseVariableMappings.containsKey(currentHead)) { + //head is not projected + projectedHead = inverseVariableMappings.get(currentHead); + projectedMembers.remove(projectedHead); + } + else { + //pick the first projected member as the new head + projectedHead = projectedMembers.get(0); + projectedMembers.remove(0); + } + } + + RowExpression finalProjectedHead = projectedHead; + projectedMembers.forEach(m -> projectedEquivalenceClassProperty.update(finalProjectedHead, m)); + } + } + return projectedEquivalenceClassProperty; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("EquivalenceClassHeads", String.join(",", equivalenceClassHeads.entrySet().stream().map(e -> e.getKey().toString() + ":" + e.getValue().toString()).collect(toImmutableList()))) + .add("EquivalenceClasses", String.join(",", equivalenceClasses.entrySet().stream().map(e -> e.getKey().toString() + ":" + e.getValue().toString()).collect(toImmutableList()))) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java new file mode 100644 index 0000000000000..847e103541831 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/Key.java @@ -0,0 +1,151 @@ +/* + * 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.properties; + +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Represents a primary or unique key constraint that holds for a final or + * intermediate result set produced by a PlanNode. + * It can also be used to represent a key requirement that must be + * satisfied by a PlanNode (e.g. distinct requirement) + */ +public class Key +{ + private final Set variables; + + /** + * A set of variable references that satisfy a primary or unique key constraint. + * + * @param variables + */ + public Key(Set variables) + { + requireNonNull(variables, "Variables is null."); + checkArgument(!variables.isEmpty(), "Variables is empty"); + this.variables = variables; + } + + /** + * Determines if a provided key requirement is satisfied by this key. + * This is true if the variables in this key are a subset of the variables in the key requirement. + * Note the this operation should be called only after using the normalize method to render + * the key and key requirement into their canonical forms using equivalence classes. + * + * @param keyRequirement + * @return True if this key satisfies the key requirement and False otherwise. + */ + public boolean keySatisifiesRequirement(Key keyRequirement) + { + requireNonNull(keyRequirement, "Key requirement must be provided."); + + //ideally this would be a simple subset operation but the "canonicalize" operation in UnliasSymbols inexplicably + //clones VariableReferenceExpression's so two references to the same outputs might be made via different objects + return variables.stream().allMatch(vk -> keyRequirement.variables.stream().anyMatch(vr -> vk.equals(vr))); + } + + /** + * Returns a canonical version of this key wherein duplicate or constant variables are removed + * and any remaining variables are replaced with their equivalence class heads. + * Note that if all key variables are bound to constants an empty result is + * returned, signaling that at most a single record is in the result set constrained + * by this key. + * + * @param equivalenceClassProperty + * @return A normalized version of this key or empty if all variables are bound to constants. + */ + public Optional normalize(EquivalenceClassProperty equivalenceClassProperty) + { + requireNonNull(equivalenceClassProperty, "Equivalence class property must be provided."); + Set unBoundVariables = new HashSet<>(); + variables.stream().forEach(v -> { + RowExpression eqHead = equivalenceClassProperty.getEquivalenceClassHead(v); + if (!(eqHead instanceof ConstantExpression)) { + unBoundVariables.add((VariableReferenceExpression) eqHead); + } + }); + if (unBoundVariables.isEmpty()) { + return Optional.empty(); + } + else { + return Optional.of(new Key(unBoundVariables)); + } + } + + /** + * Returns a projected version of this key. + * Variables in the key are mapped to output variables in the context beyond the project operation. + * If a key attribute does not have an assignment in the new attribute context, it is mapped to the assignment of + * an equivalent attribute whenever possible. For example, assume A is a key attribute and there is no new assignment + * for A. Assume further that A and B are in the same equivalence class and there is an assignment from B to B’. + * Consequently, A can be assigned to B' rather than get projected. If any of the variables are not mapped then an + * empty result is returned signaling that the key is effectively uninteresting beyond the project operation and hence is not propagated. + * + * @param inverseVariableMappings + * @return A projected version of this key or empty if any variables are not propagated. + */ + public Optional project(LogicalPropertiesImpl.InverseVariableMappingsWithEquivalence inverseVariableMappings) + { + requireNonNull(inverseVariableMappings, "Inverse variable mappings must be provided."); + Set mappedVariables = new HashSet<>(); + Optional mappedVariable; + for (VariableReferenceExpression v : variables) { + mappedVariable = inverseVariableMappings.get(v); + if (mappedVariable.isPresent()) { + mappedVariables.add(mappedVariable.get()); + continue; + } + else { + return Optional.empty(); + } + } + return Optional.of(new Key(mappedVariables)); + } + + /** + * Returns a version of this key concatenated with the provided key. + * A concatenated key results from a join operation where concatenated keys of the left and + * right join inputs form unique constraints on the join result. + * + * @param toConcatKey + * @return a version of this key concatenated with the provided key. + */ + public Key concat(Key toConcatKey) + { + requireNonNull(toConcatKey, "Key must be provided."); + Set concatenatedVariables = new HashSet(); + concatenatedVariables.addAll(this.variables); + concatenatedVariables.addAll(toConcatKey.variables); + return new Key(concatenatedVariables); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("variables", String.join(",", variables.stream().map(VariableReferenceExpression::toString).collect(Collectors.toList()))) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java new file mode 100644 index 0000000000000..c09323816b756 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/KeyProperty.java @@ -0,0 +1,216 @@ +/* + * 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.properties; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Represents a collection of primary or unique key constraints that hold for a final or + * intermediate result set produced by a PlanNode. + */ +public class KeyProperty +{ + private final Set keys; + + public KeyProperty() + { + this.keys = new HashSet<>(); + } + + public KeyProperty(Set keys) + { + this.keys = keys; + } + + /** + * Determines if one key property is more general than another. + * A key property is more general than another if it can satisfy any key requirement the other can satisfy. + * + * @param otherKeyProperty + * @return True keyProperty is more general than otherKeyProperty or False otherwise. + */ + public boolean moreGeneral(KeyProperty otherKeyProperty) + { + requireNonNull(otherKeyProperty, "otherKeyProperty is null"); + return ((keys.isEmpty() && otherKeyProperty.keys.isEmpty()) || + (otherKeyProperty.keys.stream().allMatch(k -> satisfiesKeyRequirement(k)))); + } + + /** + * Determines if this key property satisfies a key requirement. + * This is true if any of the keys in the collection satisfies the key requirement. + * + * @param keyRequirement + * @return True if keyRequirement is satisfied by this key property or False otherwise. + */ + public boolean satisfiesKeyRequirement(Key keyRequirement) + { + requireNonNull(keyRequirement, "keyRequirement is null"); + return keys.stream().anyMatch(k -> k.keySatisifiesRequirement(keyRequirement)); + } + + /** + * Adds a set of keys to this key property. + * + * @param keys + */ + public void addKeys(Set keys) + { + requireNonNull(keys, "keys is null"); + keys.stream().forEach(k -> addKey(k)); + } + + /** + * Adds the keys from the provided key property to this key property. + * + * @param keyProperty + */ + public void addKeys(KeyProperty keyProperty) + { + requireNonNull(keyProperty, "keyProperty is null"); + addKeys(keyProperty.keys); + } + + /** + * Adds a new key to this key property. + * + * @param key + */ + public void addKey(Key key) + { + requireNonNull(key, "key is null"); + addNonRedundantKey(key); + } + + /** + * Adds a key to this key property while enforcing the constraint that no + * key is redundant with respect to another. + * E.g. if {orderkey} was an existing key then the key {orderkey, orderpriority} + * would represent a redundant key. The inverse is true, an existing key + * can be removed by a new key it if is redundant with respect to the new key. + * + * @param newKey + */ + private void addNonRedundantKey(Key newKey) + { + requireNonNull(newKey, "newKey is null"); + Set removedKeys = new HashSet<>(); + for (Key key : keys) { + //if the new key >= key don't add it + if (key.keySatisifiesRequirement(newKey)) { + return; + } + + //if the new key <= key1 remove existing key. note that if this is true the new key will be added as it + //cannot be a superset of another key2 otherwise key2 <= key1 which violates the key property invariant + if (newKey.keySatisifiesRequirement(key)) { + removedKeys.add(key); + } + } + //new key not >= existing key + keys.add(newKey); + keys.removeAll(removedKeys); + } + + /** + * Reduces key property to a concise cannonical form wherein each individual key is + * reduced to a canonical form by removing redundant variables and replacing any remaining variables + * with their equivalence class heads. Moreover, no keys in the normalized key + * property are redundant with respect to the others. + * Note that if any key is fully bound to constants an empty result is + * returned, signaling that at most a single record is in the result set constrained + * by this key property. + * + * @param equivalenceClassProperty + * @return A normalized version of this key property or empty if any key is fully bound to constants. + */ + public Optional normalize(EquivalenceClassProperty equivalenceClassProperty) + { + requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + KeyProperty result = new KeyProperty(); + for (Key key : this.keys) { + Optional normalizedKey = key.normalize(equivalenceClassProperty); + if (!normalizedKey.isPresent()) { + return Optional.empty(); + } + else { + result.addKey(normalizedKey.get()); + } + } + return Optional.of(result); + } + + /** + * Returns a projected version of this key property. + * Variables in each key are mapped to output variables in the context beyond the project operation. + * It is possible that this operation projects all keys from the key property. + * + * @param inverseVariableMappings + * @return A projected version of this key property. + */ + public KeyProperty project(LogicalPropertiesImpl.InverseVariableMappingsWithEquivalence inverseVariableMappings) + { + requireNonNull(inverseVariableMappings, "inverseVariableMappings is null"); + KeyProperty result = new KeyProperty(); + keys.stream().forEach(key -> { + Optional projectedKey = key.project(inverseVariableMappings); + if (projectedKey.isPresent()) { + result.addKey(projectedKey.get()); + } + }); + return result; + } + + /** + * Empties all keys from the key property. + */ + public void empty() + { + keys.clear(); + } + + /** + * Returns a version of this key property wherein each key is concatenated with all keys in the provided key property + * A concatenated key property results from a join operation where concatenated keys of the left and + * right join inputs form unique constraints on the join result. + * + * @param toConcatKeyProp + * @return a version of this key concatenated with the provided key. + */ + public KeyProperty concat(KeyProperty toConcatKeyProp) + { + requireNonNull(toConcatKeyProp, "toConcatKeyProp is null"); + KeyProperty result = new KeyProperty(); + for (Key thisKey : this.keys) { + for (Key toConcatKey : toConcatKeyProp.keys) { + result.addKey(thisKey.concat(toConcatKey)); + } + } + return result; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("keys", String.join(",", keys.stream().map(Key::toString).collect(Collectors.toList()))) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java new file mode 100644 index 0000000000000..0948cafd41c48 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesImpl.java @@ -0,0 +1,644 @@ +/* + * 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.properties; + +import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.relation.RowExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.plan.AssignmentUtils; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.relational.FunctionResolution; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.FULL; +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.LEFT; +import static com.facebook.presto.sql.planner.plan.JoinNode.Type.RIGHT; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Provides an implementation of interface LogicalProperties along with a set + * of builders that various PlanNode's can use to compute their logical properties. + *

+ * The logical properties of a PlanNode represent properties that hold for the final + * or intermediate result produced by the PlanNode and are a function of the logical properties + * of the PlanNode's source(s) and the operation performed by the PlanNode. + * For example, and AggregationNode with a single grouping key + * would add a unique key to the properties of its input source. + *

+ * Note that for this implementation to work effectively it must sit behind the TranslateExpressions + * optimizer as it does not currently deal with original expressions. The TranslateExpressions + * functionality should ultimately be moved earlier into query compilation as opposed to + * extending this implementation with support for original expressions. + */ +public class LogicalPropertiesImpl + implements LogicalProperties +{ + private final MaxCardProperty maxCardProperty; + private final KeyProperty keyProperty; + private final EquivalenceClassProperty equivalenceClassProperty; + + public LogicalPropertiesImpl(EquivalenceClassProperty equivalenceClassProperty, MaxCardProperty maxCardProperty, KeyProperty keyProperty) + { + this.equivalenceClassProperty = requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + this.maxCardProperty = requireNonNull(maxCardProperty, "maxCardProperty is null"); + this.keyProperty = requireNonNull(keyProperty, "keyProperty is null"); + } + + private static void checkArgument(boolean condition, String message) + { + if (!condition) { + throw new IllegalArgumentException(message); + } + } + + /** + * Determines if one set of logical properties is more general than another set. + * A set of logical properties is more general than another set if they can satisfy + * any requirement the other can satisfy. See the corresponding moreGeneral method + * for each of the individual properties to get more detail on the overall semantics. + * + * @param otherLogicalProperties + * @return True if this logicalproperties is more general than otherLogicalProperties or False otherwise. + */ + private boolean isMoreGeneralThan(LogicalPropertiesImpl otherLogicalProperties) + { + requireNonNull(otherLogicalProperties, "otherLogicalProperties is null"); + return (this.maxCardProperty.moreGeneral(otherLogicalProperties.maxCardProperty) && + this.keyProperty.moreGeneral(otherLogicalProperties.keyProperty) && + this.equivalenceClassProperty.isMoreGeneralThan(otherLogicalProperties.equivalenceClassProperty)); + } + + /** + * Determines if two sets of logical properties are equivalent. + * Two sets of logical properties are equivalent if each is more general than the other. + * + * @param otherLogicalProperties + * @return True if this and otherLogicalProperties are equivalent or False otherwise. + */ + public boolean equals(LogicalPropertiesImpl otherLogicalProperties) + { + requireNonNull(otherLogicalProperties, "otherLogicalProperties is null"); + return ((this.isMoreGeneralThan(otherLogicalProperties)) && otherLogicalProperties.isMoreGeneralThan(this)); + } + + /** + * Produces the inverse mapping of the provided assignments. + * The inverse mapping is used to propagate individual properties across a project operation + * by rewriting the property's variable references to those of the + * output of the project operation as per the provided assignments. + */ + private static Map inverseVariableAssignments(Assignments assignments) + { + //TODO perhaps put this in AssignmentsUtils or ProjectUtils + requireNonNull(assignments, "assignments is null"); + Map inverseVariableAssignments = new HashMap<>(); + for (Map.Entry e : assignments.entrySet()) { + if (e.getValue() instanceof VariableReferenceExpression) { + inverseVariableAssignments.put((VariableReferenceExpression) e.getValue(), e.getKey()); + } + } + return inverseVariableAssignments; + } + + /** + * Encapsulates normalization of the key property in alignment with equivalence class property, + * and possible setting of max card property if a one record condition is detected. + * The key property is modified. Maxcard will be modified if a one record condition is detected. + */ + private static void normalizeKeyPropertyAndSetMaxCard(KeyProperty keyProperty, MaxCardProperty maxCardProperty, EquivalenceClassProperty equivalenceClassProperty) + { + if (maxCardProperty.isAtMostOne()) { + keyProperty.empty(); + return; + } + Optional normalizedKeyProperty = keyProperty.normalize(equivalenceClassProperty); + keyProperty.empty(); //add in normalized keys or set maxcard + if (normalizedKeyProperty.isPresent()) { + keyProperty.addKeys(normalizedKeyProperty.get()); + } + else { + maxCardProperty.update(1); + } + } + + @Override + public boolean isDistinct(Set keyVars) + { + requireNonNull(keyVars, "keyVars is null"); + checkArgument(!keyVars.isEmpty(), "keyVars is empty"); + return this.keyRequirementSatisfied(new Key(keyVars)); + } + + @Override + public boolean isAtMostSingleRow() + { + return this.isAtMost(1); + } + + @Override + public boolean isAtMost(long n) + { + return maxCardProperty.isAtMost(n); + } + + private boolean keyRequirementSatisfied(Key keyRequirement) + { + requireNonNull(keyRequirement, "keyRequirement is null"); + if (maxCardProperty.isAtMostOne()) { + return true; + } + Optional normalizedKeyRequirement = keyRequirement.normalize(equivalenceClassProperty); + if (normalizedKeyRequirement.isPresent()) { + return keyProperty.satisfiesKeyRequirement(keyRequirement); + } + else { + return false; + } + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("KeyProperty", keyProperty) + .add("EquivalenceClassProperty", equivalenceClassProperty) + .add("MaxCardProperty", maxCardProperty) + .toString(); + } + + /** + * This logical properties builder should be used by PlanNode's that do not + * (yet perhaps) propagate or add logical properties. For example, a GroupIdNode does + * not propagate or add logical properties. This is the PlanNode default. + */ + public static class NoPropagateBuilder + { + KeyProperty keyProperty = new KeyProperty(); + MaxCardProperty maxCardProperty = new MaxCardProperty(); + EquivalenceClassProperty equivalenceClassProperty; + + NoPropagateBuilder(FunctionResolution functionResolution) + { + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that simply + * propagate source properties without changes. For example, a SemiJoin node + * propagates the inputs of its non-filtering source without adding new properties. + * A SortNode also propagates the logical properties of its source without change. + */ + public static class PropagateBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final KeyProperty keyProperty = new KeyProperty(); + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + + PropagateBuilder(LogicalPropertiesImpl sourceProperties, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + this.sourceProperties = sourceProperties; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder is used by a TableScanNode to initialize logical properties from catalog constraints. + */ + public static class TableScanBuilder + { + List> keys; + KeyProperty keyProperty = new KeyProperty(); + MaxCardProperty maxCardProperty = new MaxCardProperty(); + EquivalenceClassProperty equivalenceClassProperty; + + TableScanBuilder(List> keys, FunctionResolution functionResolution) + { + requireNonNull(keys, "keys is null"); + this.keys = keys; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(keys.stream().map(keyCols -> new Key(keyCols)).collect(Collectors.toSet())); + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that apply predicates. + * The application of conjunct predicates that equate attributes and constants effects changes to the equivalence class property. + * When equivalence classes change, specifically when equivalence class heads change, properties that keep a canonical form + * in alignment with equivalence classes will be affected. + */ + public static class FilterBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final RowExpression predicate; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + private final KeyProperty keyProperty = new KeyProperty(); + + FilterBuilder(LogicalPropertiesImpl sourceProperties, RowExpression predicate, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(predicate, "predicate is null"); + this.sourceProperties = sourceProperties; + this.predicate = predicate; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + if (equivalenceClassProperty.update(predicate)) { + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + } + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that project their + * source properties. For example, a ProjectNode and AggregationNode project their + * source properties. The former might also reassign property variable references. + */ + public static class ProjectBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final Assignments assignments; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private KeyProperty keyProperty = new KeyProperty(); + private EquivalenceClassProperty equivalenceClassProperty; + + ProjectBuilder(LogicalPropertiesImpl sourceProperties, Assignments assignments, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(assignments, "assignments is null"); + this.sourceProperties = sourceProperties; + this.assignments = assignments; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + + //project both equivalence classes and key property + Map inverseVariableAssignments = inverseVariableAssignments(assignments); + keyProperty = keyProperty.project(new InverseVariableMappingsWithEquivalence(equivalenceClassProperty, inverseVariableAssignments)); + equivalenceClassProperty = equivalenceClassProperty.project(inverseVariableAssignments); + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that propagate their + * source properties and add a limit. For example, TopNNode and LimitNode. + */ + public static class PropagateAndLimitBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final long limit; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + private final KeyProperty keyProperty = new KeyProperty(); + + PropagateAndLimitBuilder(LogicalPropertiesImpl sourceProperties, long limit, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + this.sourceProperties = sourceProperties; + this.limit = limit; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + maxCardProperty.update(limit); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + if (maxCardProperty.isAtMostOne()) { + keyProperty.empty(); + } + return new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty); + } + } + + /** + * This logical properties builder should be used by PlanNode's that propagate their source + * properties and add a unique key. For example, an AggregationNode with a single grouping key + * propagates it's input properties and adds the grouping key attributes as a new unique key. + * The resulting properties are projected by the provided output variables. + */ + public static class AggregationBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final Key key; + private final EquivalenceClassProperty equivalenceClassProperty; + private final List outputVariables; + private final KeyProperty keyProperty = new KeyProperty(); + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final FunctionResolution functionResolution; + + AggregationBuilder(LogicalPropertiesImpl sourceProperties, Set keyVariables, List outputVariables, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(keyVariables, "keyVariables is null"); + requireNonNull(outputVariables, "outputVariables is null"); + checkArgument(!keyVariables.isEmpty(), "keyVariables is empty"); + this.sourceProperties = sourceProperties; + this.key = new Key(keyVariables); + this.outputVariables = outputVariables; + this.functionResolution = functionResolution; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + keyProperty.addKeys(sourceProperties.keyProperty); + maxCardProperty.update(sourceProperties.maxCardProperty); + equivalenceClassProperty.update(sourceProperties.equivalenceClassProperty); + //add the new key and normalize the key property unless there is a single row in the input + if (!maxCardProperty.isAtMostOne()) { + keyProperty.addKey(key); + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + } + //project the properties using the output variables to ensure only the interesting constraints propagate + ProjectBuilder projectBuilder = new ProjectBuilder(new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty), + AssignmentUtils.identityAssignments(this.outputVariables), functionResolution); + return projectBuilder.build(); + } + } + + /** + * This logical properties builder should be used by PlanNode's that propagate their source + * properties, add a unique key, and also limit the result. For example, a DistinctLimitNode. + */ + public static class DistinctLimitBuilder + { + private final LogicalPropertiesImpl sourceProperties; + private final Set keyVariables; + private final long limit; + private final List outputVariables; + private final FunctionResolution functionResolution; + + DistinctLimitBuilder(LogicalPropertiesImpl sourceProperties, Set keyVariables, Long limit, List outputVariables, FunctionResolution functionResolution) + { + requireNonNull(sourceProperties, "sourceProperties is null"); + requireNonNull(keyVariables, "keyVariables is null"); + requireNonNull(outputVariables, "outputVariables is null"); + requireNonNull(limit, "limit is null"); + checkArgument(!keyVariables.isEmpty(), "keyVariables is empty"); + this.sourceProperties = sourceProperties; + this.keyVariables = keyVariables; + this.outputVariables = outputVariables; + this.limit = limit; + this.functionResolution = functionResolution; + } + + LogicalPropertiesImpl build() + { + AggregationBuilder aggregationBuilder = new AggregationBuilder(sourceProperties, keyVariables, outputVariables, functionResolution); + PropagateAndLimitBuilder propagateAndLimitBuilder = new PropagateAndLimitBuilder(aggregationBuilder.build(), limit, functionResolution); + return propagateAndLimitBuilder.build(); + } + } + + /** + * This logical properties builder should be used by PlanNode's that join two input sources + * where both input sources contribute variables to the join output (e.g. JoinNode vs. SemiJoinNode). + * Propagation of the source properties of the join requires a sophisticated analysis of the characteristics of the join. + *

+ * Key and MaxCard Properties... + *

+ * - An inner or left join propagates the key property and maxcard property of the left source if the join is n-to-1, + * meaning that each row of the left source matches at most one row of the right source. Determining that a join is n-to-1 + * involves forming a key requirement from the equi-join attributes of the right table and querying the logical properties + * of the right table to determine if those attributes form a unique key. Semi-joins are inherently n-to1. + *

+ * - Conversely, an inner or right join can propagate the key property and maxcard property of the right source if the join is 1-to-n. + * If an inner join is 1-to-1, which is the case when it is both n-to-1 and 1-to-n, then it follows from the above that the key property + * of the join result comprises the union of the left source keys and right source keys. + *

+ * - If an inner join is instead m-to-n, meaning that it is neither n-to-1 nor 1-to-n, the key property of the join is formed by + * concatenating the left source and right source key properties. Concatenating two key properties forms a new key for every + * possible combination of keys. For example, if key property KP1 has key {A} and key {B,C} and key property KP2 has key {D} + * and key {E} the concatenating KP1 and KP2 would yield a key property with keys {A,D}, {A,E}, {B,C,D} and {B,C,E}. + * An m-to-n join propagates the product of the left source MaxCardProperty and right source MaxCardProperty if the values are both known. + *

+ * - Full outer joins do not propagate source key or maxcard properties as they can inject null rows into the result. + *

+ * EquivalenceClass Property .. + *

+ * - The equivalence class property of an inner or left join adds the equivalence classes of the left source. + *

+ * - The equivalence class property of an inner or right join adds the equivalence classes of the right source. + *

+ * - The equivalence class property of an inner join is then updated with any new equivalences resulting from the application of + * equi-join predicates, or equality conjuncts applied as filters. + *

+ * It follows from the above that inner joins combine the left and right source equivalence classes and that full outer joins do + * not propagate equivalence classes. + * Finally, the key property is normalized with the equivalence classes of the join, and both key and equivalence properties are + * projected with the join’s output attributes. + */ + public static class JoinBuilder + { + private final LogicalPropertiesImpl leftProperties; + private final LogicalPropertiesImpl rightProperties; + private final List equijoinPredicates; + private final JoinNode.Type joinType; + private final Optional filterPredicate; + private final List outputVariables; + private final MaxCardProperty maxCardProperty = new MaxCardProperty(); + private final EquivalenceClassProperty equivalenceClassProperty; + private final FunctionResolution functionResolution; + private KeyProperty keyProperty = new KeyProperty(); + + JoinBuilder(LogicalPropertiesImpl leftProperties, + LogicalPropertiesImpl rightProperties, + List equijoinPredicates, + JoinNode.Type joinType, + Optional filterPredicate, + List outputVariables, + FunctionResolution functionResolution) + { + requireNonNull(leftProperties, "leftProperties is null"); + requireNonNull(rightProperties, "rightProperties is null"); + requireNonNull(equijoinPredicates, "equijoinPredicates is null"); + requireNonNull(joinType, "joinType is null"); + requireNonNull(filterPredicate, "filterPredicate is null"); + requireNonNull(outputVariables, "outputVariables is null"); + this.leftProperties = leftProperties; + this.rightProperties = rightProperties; + this.equijoinPredicates = equijoinPredicates; + this.joinType = joinType; + this.filterPredicate = filterPredicate; + this.outputVariables = outputVariables; + this.functionResolution = functionResolution; + this.equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + } + + LogicalPropertiesImpl build() + { + // first determine if the join is n to 1 and/or 1 to n + boolean nToOne = false; + boolean oneToN = false; + Set rightJoinVariables = this.equijoinPredicates.stream().map(predicate -> predicate.getRight()).collect(Collectors.toSet()); + Set leftJoinVariables = this.equijoinPredicates.stream().map(predicate -> predicate.getLeft()).collect(Collectors.toSet()); + + //if n-to-1 inner or left join then propagate left source keys and maxcard + if ((rightProperties.maxCardProperty.isAtMostOne() || (!rightJoinVariables.isEmpty() && rightProperties.isDistinct(rightJoinVariables))) && + ((joinType == INNER || joinType == LEFT) || (joinType == FULL && leftProperties.maxCardProperty.isAtMost(1)))) { + nToOne = true; + keyProperty.addKeys(leftProperties.keyProperty); + maxCardProperty.update(leftProperties.maxCardProperty); + } + + //if 1-to-n inner or right join then propagate right source keys and maxcard + if ((leftProperties.maxCardProperty.isAtMostOne() || (!leftJoinVariables.isEmpty() && leftProperties.isDistinct(leftJoinVariables))) && + ((joinType == INNER || joinType == RIGHT) || (joinType == FULL && rightProperties.maxCardProperty.isAtMost(1)))) { + oneToN = true; + keyProperty.addKeys(rightProperties.keyProperty); + maxCardProperty.update(rightProperties.maxCardProperty); + } + + //if an n-to-m then multiply maxcards and, if inner join, concatenate keys + if (!(nToOne || oneToN)) { + maxCardProperty.update(leftProperties.maxCardProperty); + maxCardProperty.multiply(rightProperties.maxCardProperty); + if (joinType == INNER) { + keyProperty.addKeys(leftProperties.keyProperty); + keyProperty = keyProperty.concat(rightProperties.keyProperty); + } + } + + //propagate left source equivalence classes if nulls cannot be injected + if (joinType == INNER || joinType == LEFT) { + equivalenceClassProperty.update(leftProperties.equivalenceClassProperty); + } + + //propagate right source equivalence classes if nulls cannot be injected + if (joinType == INNER || joinType == RIGHT) { + equivalenceClassProperty.update(rightProperties.equivalenceClassProperty); + } + + //update equivalence classes with equijoin predicates, note that if nulls are injected, equivalence does not hold propagate + if (joinType == INNER) { + equijoinPredicates.stream().forEach(joinVariables -> equivalenceClassProperty.update(joinVariables.getLeft(), joinVariables.getRight())); + + //update equivalence classes with any residual filter predicate + if (filterPredicate.isPresent()) { + equivalenceClassProperty.update(filterPredicate.get()); + } + } + + //since we likely merged equivalence class from left and right source we will normalize the key property + normalizeKeyPropertyAndSetMaxCard(keyProperty, maxCardProperty, equivalenceClassProperty); + + //project the resulting properties by the output variables + ProjectBuilder projectBuilder = new ProjectBuilder(new LogicalPropertiesImpl(equivalenceClassProperty, maxCardProperty, keyProperty), + AssignmentUtils.identityAssignments(this.outputVariables), functionResolution); + return projectBuilder.build(); + } + } + + /** + * This is a helper method for project operations where variable references are reassigned. + * It uses equivalence classes to facilitate the reassignment. For example, if a key + * is normalized to equivalence class head X with equivalence class member Y and there is a reassignment + * of Y to YY then the variable X will be reassigned to YY assuming there is no direct + * reassignment of X to another variable reference. Useful equivalent mappings are + * determined lazily and cached. + */ + public static class InverseVariableMappingsWithEquivalence + { + private final EquivalenceClassProperty equivalenceClassProperty; + private final Map inverseMappings; + + InverseVariableMappingsWithEquivalence(EquivalenceClassProperty equivalenceClassProperty, + Map inverseMappings) + { + requireNonNull(equivalenceClassProperty, "equivalenceClassProperty is null"); + requireNonNull(inverseMappings, "inverseMappings is null"); + this.equivalenceClassProperty = equivalenceClassProperty; + this.inverseMappings = inverseMappings; + } + + private boolean containsKey(VariableReferenceExpression variable) + { + if (!inverseMappings.containsKey(variable)) { + //try to find a reverse mapping of an equivalent variable, update mappings + RowExpression head = equivalenceClassProperty.getEquivalenceClassHead(variable); + List equivalentVariables = new ArrayList<>(); + equivalentVariables.add(head); + equivalentVariables.addAll(equivalenceClassProperty.getEquivalenceClasses(head)); + for (RowExpression e : equivalentVariables) { + if (e instanceof VariableReferenceExpression && + inverseMappings.containsKey(e)) { + inverseMappings.put(variable, inverseMappings.get(e)); + break; + } + } + } + return inverseMappings.containsKey(variable); + } + + /** + * Returns a direct or equivalent mapping of the provided variable reference. + */ + public Optional get(VariableReferenceExpression variable) + { + requireNonNull(variable, "variable is null"); + if (containsKey(variable)) { + return Optional.of(inverseMappings.get(variable)); + } + else { + return Optional.empty(); + } + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java new file mode 100644 index 0000000000000..a12a3b439dcf0 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/LogicalPropertiesProviderImpl.java @@ -0,0 +1,345 @@ +/* + * 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.properties; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; +import com.facebook.presto.spi.plan.AggregationNode; +import com.facebook.presto.spi.plan.DistinctLimitNode; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.ProjectNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.TopNNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.iterative.GroupReference; +import com.facebook.presto.sql.planner.plan.AssignUniqueId; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.planner.plan.SemiJoinNode; +import com.facebook.presto.sql.planner.plan.SortNode; +import com.facebook.presto.sql.relational.FunctionResolution; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; + +/** + * Defines a suite of plan node-specific methods for the computation of logical properties. + * Supplies a default implementation that produces an empty set of logical properties, and additionally, + * a suite of plan-node specific overrides of the default implementation. The implementations leverage + * property propagation builders supplied by LogicalPropertiesImpl. The LogicalPropertiesProvider + * mechanism enables a plan node to receive its logical property compute capabilities via dependency injection. + * This is needed because the computation of logical properties requires analysis of plan node's argument expressions, + * and the code the performs this analysis must be built in presto-main as this is where expression related classes are visible. + * The property computation implementation is dynamically injected into the presto-spi and presto-main plan node method's responsible + * for computing logical properties. + */ +public class LogicalPropertiesProviderImpl + implements LogicalPropertiesProvider +{ + private final FunctionResolution functionResolution; + + public LogicalPropertiesProviderImpl(FunctionResolution functionResolution) + { + this.functionResolution = requireNonNull(functionResolution, "functionResolution is null"); + } + + /** + * Provides the logical properties for a ValuesNode. Bounds the MaxCard property to the row count. + * + * @param valuesNode + * @return The logical properties for a ValuesNode. + */ + @Override + public LogicalProperties getValuesProperties(ValuesNode valuesNode) + { + LogicalPropertiesImpl sourceProps = new LogicalPropertiesImpl.NoPropagateBuilder(functionResolution).build(); + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateAndLimitBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, valuesNode.getRows().size(), functionResolution); + return propagateAndLimitBuilder.build(); + } + + /** + * Provides the logical properties for a TableScanNode. These properties come from analysis of catalog constraints. + * The current implementation is just scaffolding that will be removed once the HMS upgrade is completed. + * + * @param tableScanNode + * @return The logical properties for a TableScanNode. + */ + @Override + public LogicalProperties getTableScanProperties(TableScanNode tableScanNode) + { + // map primary key and unique constraints from column handles to variable reference expressions + List> keys = new ArrayList<>(); + List> uniqueConstraints = tableScanNode.getTableConstraints().stream().filter(tableConstraint -> tableConstraint instanceof UniqueConstraint && (tableConstraint.isEnforced() || tableConstraint.isRely())).collect(Collectors.toList()); + if (!uniqueConstraints.isEmpty()) { + Map assignments = tableScanNode.getAssignments(); + Map inverseAssignments = assignments.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + uniqueConstraints.stream().filter(uniqueConstraint -> uniqueConstraint.getColumns().stream().allMatch(col -> inverseAssignments.containsKey(col))).forEach(uniqueConstraint -> keys.add(uniqueConstraint.getColumns().stream().map(col -> inverseAssignments.get(col)).collect(Collectors.toSet()))); + } + LogicalPropertiesImpl.TableScanBuilder logicalPropsBuilder = new LogicalPropertiesImpl.TableScanBuilder(keys, functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a FilterNode. These properties reflect the effects of applying predicates to the source properties. + * + * @param filterNode + * @return The logical properties for a FilterNode. + */ + @Override + public LogicalProperties getFilterProperties(FilterNode filterNode) + { + if (!((filterNode.getSource() instanceof GroupReference) && ((GroupReference) filterNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) filterNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.FilterBuilder logicalPropsBuilder = new LogicalPropertiesImpl.FilterBuilder(sourceProps, filterNode.getPredicate(), functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a ProjectNode. These properties are essentially a projection and reassignment of the + * variable references in the source properties. + * + * @param projectNode + * @return The logical properties for a ProjectNode. + */ + @Override + public LogicalProperties getProjectProperties(ProjectNode projectNode) + { + if (!((projectNode.getSource() instanceof GroupReference) && ((GroupReference) projectNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) projectNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.ProjectBuilder logicalPropsBuilder = new LogicalPropertiesImpl.ProjectBuilder(sourceProps, projectNode.getAssignments(), functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a JoinNode. These properties reflect the effects of combining the properties of the left and right sources. + * + * @param node An instance of JoinNode. + * @return The logical properties for a JoinNode. + */ + @Override + public LogicalProperties getJoinProperties(PlanNode node) + { + if (!(node instanceof JoinNode)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of JoinNode"); + } + + JoinNode joinNode = (JoinNode) node; + if (!((joinNode.getLeft() instanceof GroupReference) && ((GroupReference) joinNode.getLeft()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected left source PlanNode to be a GroupReference with LogicalProperties"); + } + + if (!((joinNode.getRight() instanceof GroupReference) && ((GroupReference) joinNode.getRight()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected right source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl leftProps = (LogicalPropertiesImpl) ((GroupReference) joinNode.getLeft()).getLogicalProperties().get(); + LogicalPropertiesImpl rightProps = (LogicalPropertiesImpl) ((GroupReference) joinNode.getRight()).getLogicalProperties().get(); + LogicalPropertiesImpl.JoinBuilder logicalPropsBuilder = new LogicalPropertiesImpl.JoinBuilder(leftProps, rightProps, joinNode.getCriteria(), joinNode.getType(), joinNode.getFilter(), joinNode.getOutputVariables(), functionResolution); + return logicalPropsBuilder.build(); + } + + /** + * Provides the logical properties for a SemiJoinNode. The properties of the non-filtering source are propagated without change. + * + * @param node An instance of SemiJoinNode. + * @return The logical properties for a SemiJoinNode. + */ + @Override + public LogicalProperties getSemiJoinProperties(PlanNode node) + { + if (!(node instanceof SemiJoinNode)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of SemiJoinNode"); + } + + SemiJoinNode semiJoinNode = (SemiJoinNode) node; + if (!((semiJoinNode.getSource() instanceof GroupReference) && ((GroupReference) semiJoinNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected non-filtering source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) semiJoinNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateBuilder(sourceProps, functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the logical properties for an AggregationNode. If a distinct or grouping operation is being performed then the logical properties + * reflect the addition of a unique key to the source properties. + * + * @param aggregationNode + * @return The logical properties for an AggregationNode. + */ + @Override + public LogicalProperties getAggregationProperties(AggregationNode aggregationNode) + { + if (!((aggregationNode.getSource() instanceof GroupReference) && ((GroupReference) aggregationNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + if (aggregationNode.getGroupingKeys().isEmpty() && aggregationNode.getAggregations().isEmpty()) { + throw new IllegalStateException("Aggregation node with no grouping columns and no aggregation functions"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) aggregationNode.getSource()).getLogicalProperties().get(); + if (!aggregationNode.getAggregations().isEmpty() && aggregationNode.getGroupingKeys().isEmpty()) { + //aggregation with no grouping variables, single row output + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, Long.valueOf(1), functionResolution); + return propagateBuilder.build(); + } + else { + LogicalPropertiesImpl.AggregationBuilder aggregationBuilder = new LogicalPropertiesImpl.AggregationBuilder(sourceProps, + aggregationNode.getGroupingKeys().stream().collect(Collectors.toSet()), + aggregationNode.getOutputVariables(), + functionResolution); + return aggregationBuilder.build(); + } + } + + /** + * Provides the logical properties for a AssignUniqueId. Adds the id variable as a key. + * + * @param node + * @return The logical properties for a AssignUniqueId. + */ + @Override + public LogicalProperties getAssignUniqueIdProperties(PlanNode node) + { + if (!(node instanceof AssignUniqueId)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of AssignUniqueId"); + } + + AssignUniqueId assignUniqueIdNode = (AssignUniqueId) node; + if (!((assignUniqueIdNode.getSource() instanceof GroupReference) && ((GroupReference) assignUniqueIdNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + if (assignUniqueIdNode.getIdVariable() == null) { + throw new IllegalStateException("AssignUniqueId should have an id variable"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) assignUniqueIdNode.getSource()).getLogicalProperties().get(); + Set key = new HashSet<>(); + key.add(assignUniqueIdNode.getIdVariable()); + LogicalPropertiesImpl.AggregationBuilder aggregationBuilder = new LogicalPropertiesImpl.AggregationBuilder(sourceProps, key, assignUniqueIdNode.getOutputVariables(), functionResolution); + return aggregationBuilder.build(); + } + + /** + * Provides the logical properties for a DistinctLimitNode. The resulting properties reflect the application of both a distinct and limit N to the + * source properties. The distinct aspect adds a unique key to the source properties and the limit bounds the result to N. + * + * @param distinctLimitNode + * @return The logical properties for an DistinctLimitNode. + */ + @Override + public LogicalProperties getDistinctLimitProperties(DistinctLimitNode distinctLimitNode) + { + if (!((distinctLimitNode.getSource() instanceof GroupReference) && ((GroupReference) distinctLimitNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) distinctLimitNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.DistinctLimitBuilder aggregationBuilder = new LogicalPropertiesImpl.DistinctLimitBuilder(sourceProps, + distinctLimitNode.getDistinctVariables().stream().collect(Collectors.toSet()), + distinctLimitNode.getLimit(), + distinctLimitNode.getOutputVariables(), + functionResolution); + return aggregationBuilder.build(); + } + + /** + * Provides the logical properties for a LimitNode. The properties reflect the application of a limit N to the source properties. + * + * @param limitNode + * @return The logical properties for a LimitNode. + */ + @Override + public LogicalProperties getLimitProperties(LimitNode limitNode) + { + if (!((limitNode.getSource() instanceof GroupReference) && ((GroupReference) limitNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) limitNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, limitNode.getCount(), functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the logical properties for a LimitNode. The properties reflect the application of a limit N to the source properties. + * + * @param topNNode + * @return The logical properties for a LimitNode. + */ + @Override + public LogicalProperties getTopNProperties(TopNNode topNNode) + { + if (!((topNNode.getSource() instanceof GroupReference) && ((GroupReference) topNNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected left source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) topNNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateAndLimitBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateAndLimitBuilder(sourceProps, topNNode.getCount(), functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the logical properties for a SortNode. The properties of the source are propagated without change. + * + * @param node An instance of SortNode. + * @return The logical properties for a SortNode. + */ + @Override + public LogicalProperties getSortProperties(PlanNode node) + { + if (!(node instanceof SortNode)) { + throw new IllegalArgumentException("Expected PlanNode to be instance of SortNode"); + } + + SortNode sortNode = (SortNode) node; + if (!((sortNode.getSource() instanceof GroupReference) && ((GroupReference) sortNode.getSource()).getLogicalProperties().isPresent())) { + throw new IllegalStateException("Expected source PlanNode to be a GroupReference with LogicalProperties"); + } + + LogicalPropertiesImpl sourceProps = (LogicalPropertiesImpl) ((GroupReference) sortNode.getSource()).getLogicalProperties().get(); + LogicalPropertiesImpl.PropagateBuilder propagateBuilder = new LogicalPropertiesImpl.PropagateBuilder(sourceProps, functionResolution); + return propagateBuilder.build(); + } + + /** + * Provides the default logical properties for a generic PlanNode which is essentially an empty set of properties. + * + * @return The default set of logical properties for a generic PlanNode. + */ + @Override + public LogicalProperties getDefaultProperties() + { + LogicalPropertiesImpl.NoPropagateBuilder logicalPropsBuilder = new LogicalPropertiesImpl.NoPropagateBuilder(functionResolution); + return logicalPropsBuilder.build(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java new file mode 100644 index 0000000000000..4c4d6eb980c03 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/properties/MaxCardProperty.java @@ -0,0 +1,137 @@ +/* + * 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.properties; + +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; + +/** + * Represents a provable maximum number of rows in a final or intermediate result by a PlanNode. + * For example if a unique key is fully bound to constants by predicates the maxcard property + * can be set to one. A limit or top operation might set maxcard to the value of their count argument. + * The value is unknown until determined and set. + */ +public class MaxCardProperty +{ + private Optional value; + + public MaxCardProperty() + { + this.value = Optional.empty(); + } + + public MaxCardProperty(Long value) + { + this.value = Optional.of(value); + } + + /** + * True if this maxcard is more general than another. This is the case if a) neither maxcard is known or + * b) this maxcard is known and the other maxcard is either unknown or is set to larger or equal value. + * Note that two maxcard properties are equal if each is more general than the other. + * + * @param otherMaxCardProperty + * @return True if maxCardProperty is more general than otherMaxCardProperty or False otherwise. + */ + public boolean moreGeneral(MaxCardProperty otherMaxCardProperty) + { + return (!value.isPresent() && !otherMaxCardProperty.value.isPresent()) || + (value.isPresent() && (!otherMaxCardProperty.value.isPresent() || (otherMaxCardProperty.value.get() >= value.get()))); + } + + /** + * Updates this maxcard with the provided value. Will change the current value only if the current value is unknown + * or the provided value is known and smaller than the current setting. + * + * @param value + */ + public void update(long value) + { + if (!this.value.isPresent() || this.value.get().compareTo(value) > 0) { + this.value = Optional.of(value); + } + } + + /** + * Updates this maxcard with the provided maxcard property. Will change the current value only if the current value is unknown + * or the provided value is known and smaller than the current setting. + * + * @param sourceMaxCardProperty + */ + public void update(MaxCardProperty sourceMaxCardProperty) + { + if (sourceMaxCardProperty.value.isPresent()) { + if (this.value.isPresent()) { + this.value = Optional.of(Long.min(this.value.get(), sourceMaxCardProperty.value.get())); + } + else { + this.value = Optional.of(sourceMaxCardProperty.value.get()); + } + } + } + + /** + * True if maxcard is known and set to the value 1. + * This guarantee can be used to eliminate redundant sorts, distincts, topN's, etc. + * + * @return True if maxcard is set and to the value 1. + */ + public boolean isAtMostOne() + { + return this.isAtMost(1); + } + + /** + * True if maxcard is known and is at most n. + * + * @return True if maxcard is known and is at most n. + */ + public boolean isAtMost(long n) + { + if (value.isPresent()) { + return (value.get().longValue() <= n); + } + else { + return false; + } + } + + /** + * Performs the product of this maxcard and a provided maxcard if both have known values. + * Used to compute the maxcard of a join. + * + * @param maxCardProperty + */ + public void multiply(MaxCardProperty maxCardProperty) + { + //the product of empty and anything else is empty + if (!maxCardProperty.value.isPresent()) { + this.value = Optional.empty(); + return; + } + //new value is present and so multiply the current value if it is present + if (this.value.isPresent()) { + this.value = Optional.of(Long.valueOf(this.value.get() * maxCardProperty.value.get())); + } + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("value", (this.value.isPresent() ? value.get() : "null")) + .toString(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java index 33ac080be5b20..5b7925795de9d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AssignUniqueId.java @@ -14,6 +14,8 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -73,6 +75,13 @@ public VariableReferenceExpression getIdVariable() return idVariable; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getAssignUniqueIdProperties(this); + } + @Override public R accept(InternalPlanVisitor visitor, C context) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java index b118cd7587be3..b80082849ab9c 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java @@ -15,6 +15,8 @@ import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.RowExpression; @@ -294,6 +296,13 @@ public List getSources() return ImmutableList.of(left, right); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getJoinProperties(this); + } + @Override @JsonProperty public List getOutputVariables() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java index 3e4992f73b0e7..c43d909ef399e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SemiJoinNode.java @@ -14,6 +14,8 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.relation.VariableReferenceExpression; @@ -185,6 +187,13 @@ public PlanNode replaceChildren(List newChildren) dynamicFilters); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getSemiJoinProperties(this); + } + public SemiJoinNode withDistributionType(DistributionType distributionType) { return new SemiJoinNode( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java index 8a5180dd0add1..82e4c623706e0 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/SortNode.java @@ -14,6 +14,8 @@ package com.facebook.presto.sql.planner.plan; import com.facebook.presto.spi.SourceLocation; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.OrderingScheme; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; @@ -65,6 +67,13 @@ public PlanNode getSource() return source; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getSortProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java index 19b315b51d1d6..5dac40eef2198 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java @@ -333,4 +333,9 @@ public FunctionHandle approximateSetFunction(Type valueType) { return functionAndTypeManager.lookupFunction("approx_set", fromTypes(valueType)); } + + public boolean isEqualFunction(FunctionHandle functionHandle) + { + return functionAndTypeManager.getFunctionMetadata(functionHandle).getOperatorType().map(EQUAL::equals).orElse(false); + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java new file mode 100644 index 0000000000000..201f7164b0f66 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/TestLogicalPropertyPropagation.java @@ -0,0 +1,2253 @@ +/* + * 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.common.predicate.TupleDomain; +import com.facebook.presto.matching.Captures; +import com.facebook.presto.matching.Pattern; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorId; +import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.constraints.PrimaryKeyConstraint; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.constraints.UniqueConstraint; +import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.FilterNode; +import com.facebook.presto.spi.plan.LimitNode; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.PlanNode; +import com.facebook.presto.spi.plan.TableScanNode; +import com.facebook.presto.spi.plan.ValuesNode; +import com.facebook.presto.spi.relation.ConstantExpression; +import com.facebook.presto.spi.relation.VariableReferenceExpression; +import com.facebook.presto.sql.planner.TestTableConstraintsConnectorFactory; +import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.iterative.properties.EquivalenceClassProperty; +import com.facebook.presto.sql.planner.iterative.properties.Key; +import com.facebook.presto.sql.planner.iterative.properties.KeyProperty; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesImpl; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesProviderImpl; +import com.facebook.presto.sql.planner.iterative.properties.MaxCardProperty; +import com.facebook.presto.sql.planner.iterative.rule.test.BaseRuleTest; +import com.facebook.presto.sql.planner.iterative.rule.test.RuleTester; +import com.facebook.presto.sql.planner.plan.JoinNode; +import com.facebook.presto.sql.relational.FunctionResolution; +import com.facebook.presto.testing.TestingTransactionHandle; +import com.facebook.presto.tpch.TpchColumnHandle; +import com.facebook.presto.tpch.TpchTableHandle; +import com.facebook.presto.tpch.TpchTableLayoutHandle; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableListMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slices; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.common.type.VarcharType.createVarcharType; +import static com.facebook.presto.sql.relational.Expressions.constant; +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Collections.emptyList; + +public class TestLogicalPropertyPropagation + extends BaseRuleTest +{ + private TableHandle customerTableHandle; + private TableHandle ordersTableHandle; + private TableHandle lineitemTableHandle; + + private ColumnHandle customerCustKeyColumn; + private ColumnHandle customerNationKeyColumn; + private ColumnHandle customerCommentColumn; + private ColumnHandle mktSegmentColumn; + private ColumnHandle acctBalColumn; + private ColumnHandle ordersCustKeyColumn; + private ColumnHandle ordersOrderKeyColumn; + private ColumnHandle ordersOrderPriorityColumn; + private ColumnHandle ordersCommentColumn; + private ColumnHandle shipPriorityColumn; + private ColumnHandle lineitemOrderkeyColumn; + private ColumnHandle lineitemLinenumberColumn; + private ColumnHandle lineitemExtendedPriceColumn; + + private VariableReferenceExpression customerCustKeyVariable; + private VariableReferenceExpression customerNationKeyVariable; + private VariableReferenceExpression customerCommentVariable; + private VariableReferenceExpression shipPriorityVariable; + private VariableReferenceExpression mktSegmentVariable; + private VariableReferenceExpression acctBalVariable; + private VariableReferenceExpression ordersCustKeyVariable; + private VariableReferenceExpression ordersOrderKeyVariable; + private VariableReferenceExpression ordersOrderPriorityVariable; + private VariableReferenceExpression ordersCommentVariable; + private VariableReferenceExpression lineitemOrderkeyVariable; + private VariableReferenceExpression lineitemLinenumberVariable; + private VariableReferenceExpression lineitemExtendedPriceVariable; + + private FunctionResolution functionResolution; + private LogicalPropertiesProviderImpl logicalPropertiesProvider; + + @BeforeClass + public final void setUp() + { + tester = new RuleTester(emptyList(), ImmutableMap.of(), Optional.of(1), new TestTableConstraintsConnectorFactory(1)); + ConnectorId connectorId = tester().getCurrentConnectorId(); + functionResolution = new FunctionResolution(tester.getMetadata().getFunctionAndTypeManager()); + logicalPropertiesProvider = new LogicalPropertiesProviderImpl(functionResolution); + + TpchTableHandle customerTpchTableHandle = new TpchTableHandle("customer", 1.0); + TpchTableHandle ordersTpchTableHandle = new TpchTableHandle("orders", 1.0); + TpchTableHandle lineitemTpchTableHandle = new TpchTableHandle("lineitem", 1.0); + + customerTableHandle = new TableHandle( + connectorId, + customerTpchTableHandle, + TestingTransactionHandle.create(), + Optional.of(new TpchTableLayoutHandle(customerTpchTableHandle, TupleDomain.all()))); + + ordersTableHandle = new TableHandle( + connectorId, + ordersTpchTableHandle, + TestingTransactionHandle.create(), + Optional.of(new TpchTableLayoutHandle(ordersTpchTableHandle, TupleDomain.all()))); + + lineitemTableHandle = new TableHandle( + connectorId, + lineitemTpchTableHandle, + TestingTransactionHandle.create(), + Optional.of(new TpchTableLayoutHandle(lineitemTpchTableHandle, TupleDomain.all()))); + + customerCustKeyColumn = new TpchColumnHandle("custkey", BIGINT); + customerCommentColumn = new TpchColumnHandle("comment", VARCHAR); + customerNationKeyColumn = new TpchColumnHandle("nationkey", BIGINT); + mktSegmentColumn = new TpchColumnHandle("mktsegment", VARCHAR); + acctBalColumn = new TpchColumnHandle("acctbal", DOUBLE); + ordersCustKeyColumn = new TpchColumnHandle("custkey", BIGINT); + ordersOrderKeyColumn = new TpchColumnHandle("orderkey", BIGINT); + ordersOrderPriorityColumn = new TpchColumnHandle("orderpriority", BIGINT); + shipPriorityColumn = new TpchColumnHandle("shippriority", INTEGER); + ordersCommentColumn = new TpchColumnHandle("comment", VARCHAR); + lineitemOrderkeyColumn = new TpchColumnHandle("orderkey", BIGINT); + lineitemLinenumberColumn = new TpchColumnHandle("linenumber", BIGINT); + lineitemExtendedPriceColumn = new TpchColumnHandle("extendedprice", DOUBLE); + + customerCustKeyVariable = new VariableReferenceExpression(Optional.empty(), "c_custkey", BIGINT); + customerNationKeyVariable = new VariableReferenceExpression(Optional.empty(), "nationkey", BIGINT); + customerCommentVariable = new VariableReferenceExpression(Optional.empty(), "c_comment", VARCHAR); + mktSegmentVariable = new VariableReferenceExpression(Optional.empty(), "c_mktsegment", VARCHAR); + acctBalVariable = new VariableReferenceExpression(Optional.empty(), "c_acctbal", DOUBLE); + ordersCustKeyVariable = new VariableReferenceExpression(Optional.empty(), "o_custkey", BIGINT); + ordersOrderKeyVariable = new VariableReferenceExpression(Optional.empty(), "o_orderkey", BIGINT); + ordersOrderPriorityVariable = new VariableReferenceExpression(Optional.empty(), "o_orderpriority", VARCHAR); + shipPriorityVariable = new VariableReferenceExpression(Optional.empty(), "o_shippriority", INTEGER); + ordersCommentVariable = new VariableReferenceExpression(Optional.empty(), "o_comment", DOUBLE); + lineitemOrderkeyVariable = new VariableReferenceExpression(Optional.empty(), "l_orderkey", BIGINT); + lineitemLinenumberVariable = new VariableReferenceExpression(Optional.empty(), "l_linenumber", BIGINT); + lineitemExtendedPriceVariable = new VariableReferenceExpression(Optional.empty(), "l_extendedprice", DOUBLE); + } + + @Test + void testValuesNodeLogicalProperties() + { + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression c = p.variable("c"); + return p.aggregation(builder -> builder + .singleGroupingSet(c) + .source(p.values(1, c))); + }) + .matches(expectedLogicalProperties); + + //Values has more than one row. + VariableReferenceExpression a = new VariableReferenceExpression(Optional.empty(), "a", BIGINT); + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(3L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.values( + ImmutableList.of(a), + ImmutableList.of(ImmutableList.of(constant(1L, BIGINT)), + ImmutableList.of(constant(2L, BIGINT)), + ImmutableList.of(constant(3L, BIGINT))))) + .matches(expectedLogicalProperties); + } + + @Test + public void testTableScanNodeLogicalProperties() + { + // "custkey" should be a key in the result of TableScan(customer) + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))) + .matches(expectedLogicalProperties); + + // add an additional unique constraint on customer (comment, nationkey)column + Set commentcolumnSet = new HashSet<>(); + commentcolumnSet.add(customerCommentColumn); + UniqueConstraint commentConstraint = new UniqueConstraint<>(commentcolumnSet, true, true); + List> customerConstraints = new ArrayList<>(tester().getTableConstraints(customerTableHandle)); + customerConstraints.add(commentConstraint); + + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable)), new Key(ImmutableSet.of(customerCommentVariable))))); + List> finalCustomerConstraints = customerConstraints; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints)) + .matches(expectedLogicalProperties); + + //TEST: the previous test but there is no assigment for the comment column + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints)) + .matches(expectedLogicalProperties); + + //TEST: add a superfulous unique constraint on the (custkey, comment) combination + Set custkeyCommentColumnSet = new HashSet<>(); + custkeyCommentColumnSet.add(customerCustKeyColumn); + custkeyCommentColumnSet.add(customerCommentColumn); + UniqueConstraint custkeyCommentConstraint = new UniqueConstraint<>(custkeyCommentColumnSet, true, true); + customerConstraints = new ArrayList<>(tester().getTableConstraints(customerTableHandle)); + customerConstraints.add(custkeyCommentConstraint); + + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + List> finalCustomerConstraints1 = customerConstraints; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, + customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints1)) + .matches(expectedLogicalProperties); + + //Define a table with key (A,B) but only give a table scan mapping for A (B). The key property of the table scan should be empty. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of())); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + ImmutableList.of(custkeyCommentConstraint))) + .matches(expectedLogicalProperties); + + // INVARIANT: define a table with primary key (A) and unique key (A,B) and ensure that the table scan key property only has key (A) (both A and B should have mappings) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, + customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + finalCustomerConstraints1)) + .matches(expectedLogicalProperties); + + // INVARIANT: define a table with primary key (A,B) and unique key (A) and ensure that the table scan key property only has key (A) (both A and B should have mappings) + PrimaryKeyConstraint custkeyCommentPK = new PrimaryKeyConstraint<>("primarykey", custkeyCommentColumnSet, true, true); + UniqueConstraint custkeyUniqueConstraint = new UniqueConstraint<>(ImmutableSet.of(customerCustKeyColumn), true, true); + + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, + customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + ImmutableList.of(custkeyCommentPK, custkeyUniqueConstraint))) + .matches(expectedLogicalProperties); + } + + @Test + void testFilterNodeLogicalProperties() + { + ConstantExpression constExpr = new ConstantExpression(100L, BIGINT); + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(customerCustKeyVariable, constExpr); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + // Primary key will be propagated till the FilterNode, where maxCardProperty will be set to 1 + // and KeyProperty will be cleared. Equivalence class property reflects the predicate "custkey = 100". + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerCustKeyVariable); + return p.filter(p.rowExpression("c_custkey = BIGINT '100'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(acctBalVariable), + ImmutableMap.of(acctBalVariable, acctBalColumn, + customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with keys (A) and (B,C) and apply predicate A=B and ensure that the filter key property only has key (A) + ColumnHandle colA = new TpchColumnHandle("A", BIGINT); + ColumnHandle colB = new TpchColumnHandle("B", BIGINT); + ColumnHandle colC = new TpchColumnHandle("C", BIGINT); + + PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA), true, true); + UniqueConstraint uniqueConstraint = new UniqueConstraint<>(ImmutableSet.of(colB, colC), true, true); + List> tableConstraints = ImmutableList.of(primaryKeyConstraint, uniqueConstraint); + + VariableReferenceExpression varA = new VariableReferenceExpression(Optional.empty(), "A", BIGINT); + VariableReferenceExpression varB = new VariableReferenceExpression(Optional.empty(), "B", BIGINT); + VariableReferenceExpression varC = new VariableReferenceExpression(Optional.empty(), "C", BIGINT); + + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, varB); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(varA))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = B"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with keys (A,C) and (B,C) and apply predicate A=constant and ensure that the filter key property has only has one key (C) + PrimaryKeyConstraint primaryKeyConstraint1 = new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA, colC), true, true); + UniqueConstraint uniqueConstraint1 = new UniqueConstraint<>(ImmutableSet.of(colB, colC), true, true); + List> tableConstraints1 = ImmutableList.of(primaryKeyConstraint1, uniqueConstraint1); + + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, constant(100L, BIGINT)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(varC))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = BIGINT '100'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with key (A,B) and apply predicates A=constant1 and B=constant2 ensure that the filter has maxcard=1 and key property is empty + + List> tableConstraints2 = ImmutableList.of(new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA, colB), true, true)); + + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, constant(100L, BIGINT)); + equivalenceClasses.update(varB, constant(50L, BIGINT)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = BIGINT '100' AND B = BIGINT '50'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints2)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: define a table with key (A,B) and apply predicates A=constant and A=B ensure that the filter has maxcard=1 and key property is empty + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(varA, varB); + equivalenceClasses.update(varA, constant(100L, BIGINT)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.filter(p.rowExpression("A = B AND A = BIGINT '100'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB, varC, colC), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints2)); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testProjectNodeLogicalProperties() + { + VariableReferenceExpression projectedCustKeyVariable = new VariableReferenceExpression(Optional.empty(), "newcustkey", BIGINT); + Assignments assignments = Assignments.builder().put(projectedCustKeyVariable, customerCustKeyVariable).build(); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedCustKeyVariable))))); + + // Test Logical properties generated for the project node + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.project(assignments, p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))) + .matches(expectedLogicalProperties); + + //TableScan has key property (A,B). Project only has mapping A->A' and hence result key property should be empty. + ColumnHandle colA = new TpchColumnHandle("A", BIGINT); + ColumnHandle colB = new TpchColumnHandle("B", BIGINT); + VariableReferenceExpression varA = new VariableReferenceExpression(Optional.empty(), "A", BIGINT); + VariableReferenceExpression varB = new VariableReferenceExpression(Optional.empty(), "B", BIGINT); + VariableReferenceExpression projectedVarA = new VariableReferenceExpression(Optional.empty(), "A1", BIGINT); + List> tableConstraints = ImmutableList.of(new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA, colB), true, true)); + Assignments assignments1 = Assignments.builder().put(projectedVarA, varA).build(); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.project(assignments1, p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints)); + }) + .matches(expectedLogicalProperties); + + //TableScan key property has key (A), Filter applies predicate A=B, Project only has a mapping B->B'. Project should have key property with (B'). + List> tableConstraints1 = ImmutableList.of(new PrimaryKeyConstraint<>("primarykey", ImmutableSet.of(colA), true, true)); + VariableReferenceExpression projectedA = new VariableReferenceExpression(Optional.empty(), "A1", BIGINT); + Assignments assignments2 = Assignments.builder().put(projectedA, varA).build(); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedA))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + return p.project(assignments2, + p.filter(p.rowExpression("A = B"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA), + ImmutableMap.of(varA, colA, varB, colB), + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // INVARIANT Filter creates multiple equivalence classes e.g. (A, B, C) (D, E, F). Test various cases where + // all or only some of these have mappings. These should include cases where the equivalence class heads are + // projected out, members are projected out, and also cases where the entire equivalence class is projected out. + ColumnHandle colC = new TpchColumnHandle("C", BIGINT); + ColumnHandle colD = new TpchColumnHandle("D", BIGINT); + ColumnHandle colE = new TpchColumnHandle("E", BIGINT); + ColumnHandle colF = new TpchColumnHandle("F", BIGINT); + VariableReferenceExpression varC = new VariableReferenceExpression(Optional.empty(), "C", BIGINT); + VariableReferenceExpression varD = new VariableReferenceExpression(Optional.empty(), "D", BIGINT); + VariableReferenceExpression varE = new VariableReferenceExpression(Optional.empty(), "E", BIGINT); + VariableReferenceExpression varF = new VariableReferenceExpression(Optional.empty(), "F", BIGINT); + VariableReferenceExpression projectedB = new VariableReferenceExpression(Optional.empty(), "B1", BIGINT); + VariableReferenceExpression projectedC = new VariableReferenceExpression(Optional.empty(), "C1", BIGINT); + VariableReferenceExpression projectedD = new VariableReferenceExpression(Optional.empty(), "D1", BIGINT); + VariableReferenceExpression projectedE = new VariableReferenceExpression(Optional.empty(), "E1", BIGINT); + VariableReferenceExpression projectedF = new VariableReferenceExpression(Optional.empty(), "F1", BIGINT); + + Map scanAssignments = + new ImmutableMap.Builder() + .put(varA, colA) + .put(varB, colB) + .put(varC, colC) + .put(varD, colD) + .put(varE, colE) + .put(varF, colF).build(); + + Assignments projectAssignments = Assignments.builder() + .put(projectedA, varA) + .put(projectedB, varB) + .put(projectedC, varC) + .put(projectedD, varD) + .put(projectedE, varE) + .put(projectedF, varF) + .build(); + + // A = B and B = C and D = E and E = F + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(projectedA, projectedB); + equivalenceClassProperty.update(projectedB, projectedC); + equivalenceClassProperty.update(projectedD, projectedE); + equivalenceClassProperty.update(projectedE, projectedF); + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClassProperty, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedA))))); + + // ProjectNode projects all variables used in the filter. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // ProjectNode projects only equivalence class heads(A, D). Equivalence classes should be empty. KeyProperty + // should be set to A1. + Assignments projectAssignments1 = Assignments.builder() + .put(projectedA, varA) + .put(projectedD, varD) + .build(); + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedA))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments1, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // ProjectNode projects only equivalence class members(B,C,E,F). Equivalence classes should have (B,C), (E,F). + // KeyProperty should have B1 + Assignments projectAssignments2 = Assignments.builder() + .put(projectedB, varB) + .put(projectedC, varC) + .put(projectedE, varE) + .put(projectedF, varF) + .build(); + EquivalenceClassProperty equivalenceClassProperty1 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty1.update(projectedB, projectedC); + equivalenceClassProperty1.update(projectedE, projectedF); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClassProperty1, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(projectedB))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments2, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + + // ProjectNode projects only equivalence class members(E,F). Equivalence classes should have (E,F). + // KeyProperty should become empty since A,B,C are removed from projection. + Assignments projectAssignments3 = Assignments.builder() + .put(projectedE, varE) + .put(projectedF, varF) + .build(); + EquivalenceClassProperty equivalenceClassProperty2 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty2.update(projectedE, projectedF); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClassProperty2, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(varA); + p.variable(varB); + p.variable(varC); + p.variable(varD); + p.variable(varE); + p.variable(varF); + + return p.project(projectAssignments3, + p.filter(p.rowExpression("A = B AND B = C AND D = E AND E = F"), + p.tableScan( + customerTableHandle, + ImmutableList.of(varA, varB, varC, varD, varE, varF), + scanAssignments, + TupleDomain.none(), + TupleDomain.none(), + tableConstraints1))); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testJoinNodeLogicalProperties() + { + // TEST: n to 1 inner join between orders and customers with limit 5 on left table. + // orders key property, maxcard=5 and equivalence classes(o_custkey=c_custkey), (shippriority=10) and + // (mktsegment='BUILDING') should be propagated. + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + equivalenceClasses.update(shipPriorityVariable, constant(10L, INTEGER)); + equivalenceClasses.update(mktSegmentVariable, constant(Slices.utf8Slice("BUILDING"), createVarcharType(8))); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.limit(5, ordersTableScan), + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 inner join between orders and customers with limit 1 on left table. maxcard=1 and equivalence + // classes(o_custkey=c_custkey),(shippriority=10) and (mktsegment='BUILDING') should be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.limit(1, ordersTableScan), + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n inner join between customers and orders with limit(6) on the right table. + // orders key property and maxcard=6 and equivalence classes(o_custkey=c_custkey),(shippriority=10) and + // (mktsegment='BUILDING') should be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.limit(6, ordersTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n inner join between customers and orders with limit(1) on the right table. + // Only maxcard=1 should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty(ImmutableSet.of())); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + p.variable(shipPriorityVariable); + p.variable(mktSegmentVariable); + return p.join(JoinNode.Type.INNER, + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), customerTableScan), + p.limit(1, ordersTableScan), + p.rowExpression("o_shippriority = 10"), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 left join between orders and customers with limit(7) on the left table. + // orders keys and maxcard=7 are propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(7L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.LEFT, p.limit(7, ordersTableScan), customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 left join between orders and customers with limit on right table. + // orders keys are propagated. Maxcard should not be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.LEFT, ordersTableScan, p.limit(8, customerTableScan), + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n right join between customers and orders. Limit(9) on the right table. + // orders keys are propagated and maxcard=9 should be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(9L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.RIGHT, customerTableScan, p.limit(9, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n right join between customers and orders. Limit(10) on the left table. + // orders keys are propagated. maxcard should not be propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.RIGHT, p.limit(10, customerTableScan), ordersTableScan, + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n left join between customers and orders - no keys are propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.LEFT, customerTableScan, ordersTableScan, + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m inner join between customers and orders - concatenated key (orderkey, custkey) should get propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, customerTableScan, ordersTableScan, emptyList(), ImmutableList.of(ordersOrderKeyVariable, customerCustKeyVariable, ordersCustKeyVariable), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m inner join between customers and orders and limit 11 left table and limit 12 on right table. + // concatenated key (orderkey, custkey) should get propagated. Maxcard should be maxCardLeft * maxCardRight = 132. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(132L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.limit(11, customerTableScan), p.limit(12, ordersTableScan), emptyList(), ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, customerCustKeyVariable), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + //test m to n cases where there are multiple keys in the left and right tables to concatenate e.g. add unique keys customer.comment and orders.comment + List> customerTableConstraints = new ArrayList<>(tester().getTableConstraints(customerTableHandle)); + customerTableConstraints.add(new UniqueConstraint<>(ImmutableSet.of(customerCommentColumn), true, true)); + + List> orderTableConstraints = new ArrayList<>(tester().getTableConstraints(ordersTableHandle)); + orderTableConstraints.add(new UniqueConstraint<>(ImmutableSet.of(ordersCommentColumn), true, true)); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable)), + new Key(ImmutableSet.of(customerCommentVariable, ordersCommentVariable)), + new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCommentVariable)), + new Key(ImmutableSet.of(customerCustKeyVariable, ordersCommentVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, customerCommentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, customerCommentVariable, customerCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + customerTableConstraints); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, ordersCommentVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + ordersCommentVariable, ordersCommentColumn), + TupleDomain.none(), + TupleDomain.none(), + orderTableConstraints); + + return p.join(JoinNode.Type.INNER, customerTableScan, ordersTableScan, emptyList(), + ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, ordersCommentVariable, + customerCustKeyVariable, customerCommentVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m inner join between values(1) and values(1) - maxcard(1) should get propagated. + VariableReferenceExpression c1 = new VariableReferenceExpression(Optional.empty(), "c1", BIGINT); + VariableReferenceExpression c2 = new VariableReferenceExpression(Optional.empty(), "c2", BIGINT); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values1 = p.values(1, c1); + ValuesNode values2 = p.values(1, c2); + return p.join(JoinNode.Type.INNER, values1, values2, emptyList(), ImmutableList.of(c1, c2), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m full join between values(1) and values(1) - maxcard(1) should get propagated. + VariableReferenceExpression c3 = new VariableReferenceExpression(Optional.empty(), "c1", BIGINT); + VariableReferenceExpression c4 = new VariableReferenceExpression(Optional.empty(), "c2", BIGINT); + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values1 = p.values(1, c3); + ValuesNode values2 = p.values(1, c4); + return p.join(JoinNode.Type.FULL, values1, values2, emptyList(), ImmutableList.of(c3, c4), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n full join between customers and orders - nothing should get propagated. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.FULL, customerTableScan, ordersTableScan, emptyList(), ImmutableList.of(customerCustKeyVariable, ordersOrderKeyVariable), Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 full join between customers and orders with limit(12) on left and and limit(10) on right table. + // The product of the maxcards 120 should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(120L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.FULL, p.limit(12, customerTableScan), p.limit(10, ordersTableScan), + ImmutableList.of(new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)), + ImmutableList.of(ordersOrderKeyVariable, customerCustKeyVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m full join between customers and orders with maxcard 2 on left and unknown maxcard on right table. + // Concatenated keys and a maxcard of unknown should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.limit(2, customerTableScan), ordersTableScan, emptyList(), + ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, + customerCustKeyVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: n to m full join between customers and orders with maxcard 2 on left and unknown maxcard on right table. + // Concatenated keys and a maxcard of unknown should get propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, ordersTableScan, p.limit(2, customerTableScan), emptyList(), + ImmutableList.of(ordersOrderKeyVariable, ordersCustKeyVariable, + customerCustKeyVariable), + Optional.empty()); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to 1 inner join between values(1) and customers - maxcard(1) is propagated + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + VariableReferenceExpression c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + equivalenceClasses.update(c, customerCustKeyVariable); + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC1 = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC1); + + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.INNER, values, customerTableScan, + new JoinNode.EquiJoinClause(finalC1, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to 1 inner join between customers and values(1) - maxcard(1) is propagated + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + equivalenceClasses.update(c, customerCustKeyVariable); + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC); + + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.INNER, customerTableScan, values, + new JoinNode.EquiJoinClause(customerCustKeyVariable, finalC)); + }) + .matches(expectedLogicalProperties); + + // TEST: 1 to n full join between customers and values(1) where n=1 - maxcard(1) is propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC2 = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC2); + + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.FULL, customerTableScan, values, + new JoinNode.EquiJoinClause(customerCustKeyVariable, finalC2)); + }) + .matches(expectedLogicalProperties); + + // TEST: n to 1 full join between values(1) and customers where n=1 - maxcard(1) is propagated + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + VariableReferenceExpression finalC3 = c; + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode values = p.values(1, finalC3); + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.join(JoinNode.Type.FULL, values, customerTableScan, + new JoinNode.EquiJoinClause(finalC3, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + // Three table join. Key (l_orderkey, l_linenumber), maxCard=6 and equivalence classes (o_orderkey,l_orderkey) and + // (o_custkey, c_custkey) should be propagated. + equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(customerCustKeyVariable, ordersCustKeyVariable); + equivalenceClasses.update(ordersOrderKeyVariable, lineitemOrderkeyVariable); + + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(lineitemLinenumberVariable, ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + TableScanNode lineitemTableScan = p.tableScan( + lineitemTableHandle, + ImmutableList.of(lineitemLinenumberVariable, lineitemOrderkeyVariable), + ImmutableMap.of(lineitemLinenumberVariable, lineitemLinenumberColumn, + lineitemOrderkeyVariable, lineitemOrderkeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(lineitemTableHandle)); + + JoinNode customerOrderJoin = p.join(JoinNode.Type.INNER, + customerTableScan, + p.limit(6, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + + return p.join(JoinNode.Type.INNER, + customerOrderJoin, + lineitemTableScan, + new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderkeyVariable)); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testSemiJoinNodeLogicalProperties() + { + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + // Test Logical properties generated for semi join node. It just propagates its non-filtering source's properties. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.semiJoin(ordersCustKeyVariable, customerCustKeyVariable, ordersOrderKeyVariable, + Optional.empty(), Optional.empty(), ordersTableScan, customerTableScan); + }) + .matches(expectedLogicalProperties); + + //source table is 1-tuple + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + ValuesNode semiJoinSource = p.values(1, ordersCustKeyVariable); + TableScanNode semiJoinFilteringSource = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.semiJoin(ordersCustKeyVariable, customerCustKeyVariable, ordersOrderKeyVariable, + Optional.empty(), Optional.empty(), semiJoinSource, semiJoinFilteringSource); + }) + .matches(expectedLogicalProperties); + + //maxcard derived from limit propagates semijoin + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(ordersOrderPriorityVariable, constant(Slices.utf8Slice("URGENT"), createVarcharType(6))); + + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode semiJoinFilteringSource = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + p.variable(ordersOrderPriorityVariable); + LimitNode semiJoinSource = p.limit(5, + p.filter(p.rowExpression("o_orderpriority = 'URGENT'"), + p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + ordersOrderPriorityVariable, ordersOrderPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)))); + + return p.semiJoin(ordersCustKeyVariable, customerCustKeyVariable, ordersOrderKeyVariable, + Optional.empty(), Optional.empty(), semiJoinSource, semiJoinFilteringSource); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testAggregationNodeLogicalProperties() + { + // Aggregation node adds new key (nationkey) + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable) + .source(p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())))) + .matches(expectedLogicalProperties); + + //INVARIANT: Grouping on (nationkey, custkey) but (custkey) is already a key. So grouping result should have only (custkey) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.aggregation(builder -> builder + .singleGroupingSet(customerCustKeyVariable, customerNationKeyVariable) + .source(p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))))) + .matches(expectedLogicalProperties); + + //INVARIANT. Group by nationkey Filter binds nationkey to a constant before grouping. Result should have maxcard=1; + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(customerNationKeyVariable, constant(20L, BIGINT)); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty, + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable) + .source(p.filter(p.rowExpression("nationkey = BIGINT '20'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Group on (nationkey, mktsegment) and after first binding "mktsegment" to a constant. The grouping result should have key (nationkey) + EquivalenceClassProperty equivalenceClassProperty1 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty1.update(mktSegmentVariable, constant(Slices.utf8Slice("BUILDING"), createVarcharType(8))); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty1, + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(mktSegmentVariable); + return p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable, mktSegmentVariable) + .source(p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle))))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Group by with aggregate functions but no grouping columns. Maxard should be 1 and no keys propagated. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.aggregation(builder -> builder + .addAggregation(p.variable("count_nk"), p.rowExpression("count(nationkey)")) + .globalGrouping() + .source(p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Maxcard is set to 1 prior to group by with grouping columns. Maxard of group by should be 1 and no keys propagated. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.aggregation(builder -> builder + .singleGroupingSet(customerNationKeyVariable) + .source(p.limit(1, p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())))); + }) + .matches(expectedLogicalProperties); + + // Test propagation of equivalence classes through aggregation. + // None of the equivalence classes from aggregation's source node should be propagated since none of the + // members are projected by the aggregation node. + // Key property (shippriority, linenumber) which form the group by keys and maxcard=6 should be propagated. + + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(shipPriorityVariable, lineitemLinenumberVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + TableScanNode lineitemTableScan = p.tableScan( + lineitemTableHandle, + ImmutableList.of(lineitemLinenumberVariable, lineitemOrderkeyVariable), + ImmutableMap.of(lineitemLinenumberVariable, lineitemLinenumberColumn, + lineitemOrderkeyVariable, lineitemOrderkeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(lineitemTableHandle)); + + JoinNode customerOrderJoin = p.join(JoinNode.Type.INNER, + customerTableScan, + p.limit(6, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + + p.variable(lineitemExtendedPriceVariable); + return p.aggregation(builder -> builder + .addAggregation(p.variable("sum_price", DOUBLE), p.rowExpression("sum(l_extendedprice)")) + .singleGroupingSet(lineitemLinenumberVariable, shipPriorityVariable) + .source(p.join(JoinNode.Type.INNER, + customerOrderJoin, + lineitemTableScan, + new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderkeyVariable)))); + }) + .matches(expectedLogicalProperties); + + // A variation to the above case, where in groupby keys are (l_lineitem,o_orderkey,shippriority). Since + // (o_orderkey, l_lineitem) are already a key, the key should be normalized to have only (o_orderkey, l_lineitem). + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable, lineitemLinenumberVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable, mktSegmentVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable, shipPriorityVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, + ordersOrderKeyVariable, ordersOrderKeyColumn, + shipPriorityVariable, shipPriorityColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + TableScanNode lineitemTableScan = p.tableScan( + lineitemTableHandle, + ImmutableList.of(lineitemLinenumberVariable, lineitemOrderkeyVariable), + ImmutableMap.of(lineitemLinenumberVariable, lineitemLinenumberColumn, + lineitemOrderkeyVariable, lineitemOrderkeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(lineitemTableHandle)); + + JoinNode customerOrderJoin = p.join(JoinNode.Type.INNER, + customerTableScan, + p.limit(6, ordersTableScan), + new JoinNode.EquiJoinClause(customerCustKeyVariable, ordersCustKeyVariable)); + + p.variable(lineitemExtendedPriceVariable); + return p.aggregation(builder -> builder + .addAggregation(p.variable("sum_price", DOUBLE), p.rowExpression("sum(l_extendedprice)")) + .singleGroupingSet(lineitemLinenumberVariable, ordersOrderKeyVariable, shipPriorityVariable) + .source(p.join(JoinNode.Type.INNER, + customerOrderJoin, + lineitemTableScan, + new JoinNode.EquiJoinClause(ordersOrderKeyVariable, lineitemOrderkeyVariable)))); + }) + .matches(expectedLogicalProperties); + } + + @Test + void testAssignUniqueIdNodeLogicalProperties() + { + VariableReferenceExpression c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + VariableReferenceExpression unique = new VariableReferenceExpression(Optional.empty(), "unique", BIGINT); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(unique))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.assignUniqueId(unique, + p.values(5, c))) + .matches(expectedLogicalProperties); + } + + @Test + void testDistinctLimitNodeLogicalProperties() + { + VariableReferenceExpression c = new VariableReferenceExpression(Optional.empty(), "c", BIGINT); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(3L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(c))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.distinctLimit(3, ImmutableList.of(c), p.values(5, c))) + .matches(expectedLogicalProperties); + + //Tests where where DistinctLimit adds a key! Mirror the aggregation tests. + + // DistinctLimit node adds new key (nationkey) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.distinctLimit(5, ImmutableList.of(customerNationKeyVariable), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList()))) + .matches(expectedLogicalProperties); + + //INVARIANT: DistinctLimit on (nationkey, custkey) but (custkey) is already a key. So grouping result should have only (custkey) + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.distinctLimit(6, ImmutableList.of(customerCustKeyVariable, customerNationKeyVariable), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))) + .matches(expectedLogicalProperties); + + //INVARIANT. DistinctLimit with nationkey as distinct symbol. + // Filter binds nationkey to a constant before grouping. Result should have maxcard=1; + EquivalenceClassProperty equivalenceClassProperty = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty.update(customerNationKeyVariable, constant(20L, BIGINT)); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty, + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.distinctLimit(6, ImmutableList.of(customerNationKeyVariable), + p.filter(p.rowExpression("nationkey = BIGINT '20'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList()))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. DistinctLimit with (nationkey, mktsegment) as symbols and after first binding "mktsegment" to a constant. + // The grouping result should have key (nationkey) + EquivalenceClassProperty equivalenceClassProperty1 = new EquivalenceClassProperty(functionResolution); + equivalenceClassProperty1.update(mktSegmentVariable, constant(Slices.utf8Slice("BUILDING"), createVarcharType(8))); + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClassProperty1, + new MaxCardProperty(7L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerNationKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(mktSegmentVariable); + return p.distinctLimit(7, ImmutableList.of(customerNationKeyVariable, mktSegmentVariable), + p.filter(p.rowExpression("c_mktsegment = 'BUILDING'"), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn, mktSegmentVariable, mktSegmentColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)))); + }) + .matches(expectedLogicalProperties); + + //INVARIANT. Maxcard is set to 1 prior to distinct limit. Maxard of distinct limit should be 1 and no keys propagated. + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.distinctLimit(8, ImmutableList.of(customerNationKeyVariable), + p.limit(1, p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList()))); + }) + .matches(expectedLogicalProperties); + + //test cases where the DistinctLimit count is 1 and results in maxcard 1 + expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + p.variable(customerNationKeyVariable); + return p.distinctLimit(1, ImmutableList.of(customerNationKeyVariable), + p.tableScan( + customerTableHandle, + ImmutableList.of(customerNationKeyVariable), + ImmutableMap.of(customerNationKeyVariable, customerNationKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + emptyList())); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testLimitNodeLogicalProperties() + { + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + // Test Logical properties generated for the Limit node. It updates the MaxCardProperty from the source properties. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + JoinNode ordersCustomerJoin = p.join(JoinNode.Type.INNER, ordersTableScan, customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + + return p.limit(6, ordersCustomerJoin); + }) + .matches(expectedLogicalProperties); + + //Do a variation of the previous test where the innerjoin(TopN(orders), customer). The maxcard(5) should propagate. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.limit(5, ordersTableScan), customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN and/or Limit comes along and tries to set it to N>K. Should still be set to K. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.limit(10, p.values(5, p.variable("c")))) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN and/or Limit comes along and tries to set it to N p.limit(6, p.values(10, p.variable("c")))) + .matches(expectedLogicalProperties); + + //INVARIANT: TableScan with key (A) and TopN and/or Limit sets result N=1. Key property should be emptied. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.limit(1, customerTableScan); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testTopNNodeLogicalProperties() + { + //just duplicate the comprehensive limit tests but also do negative tests for the case where TopN is not final + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(ordersCustKeyVariable, customerCustKeyVariable); + + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(6L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + // Test Logical properties generated for the TopN node. It updates the MaxCardProperty from the source properties. + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + JoinNode ordersCustomerJoin = p.join(JoinNode.Type.INNER, ordersTableScan, customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + + return p.topN(6, ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ordersCustomerJoin); + }) + .matches(expectedLogicalProperties); + + //Variation of the previous test where the innerjoin(TopN(orders), customer). The maxcard(5) should propagate. + expectedLogicalProperties = new LogicalPropertiesImpl( + equivalenceClasses, + new MaxCardProperty(5L), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(ordersOrderKeyVariable))))); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + TableScanNode ordersTableScan = p.tableScan( + ordersTableHandle, + ImmutableList.of(ordersCustKeyVariable, ordersOrderKeyVariable), + ImmutableMap.of(ordersCustKeyVariable, ordersCustKeyColumn, ordersOrderKeyVariable, ordersOrderKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(ordersTableHandle)); + + return p.join(JoinNode.Type.INNER, p.topN(5, ImmutableList.of(ordersCustKeyVariable), ordersTableScan), customerTableScan, + new JoinNode.EquiJoinClause(ordersCustKeyVariable, customerCustKeyVariable)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN comes along and tries to set it to N>K. Should still be set to K. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(5L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression var = p.variable("c"); + return p.topN(10, ImmutableList.of(var), p.values(5, var)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: maxcard is set to K (by Values or Filter) and TopN and/or Limit comes along and tries to set it to N { + VariableReferenceExpression var = p.variable("c"); + return p.topN(6, ImmutableList.of(var), p.values(10, var)); + }) + .matches(expectedLogicalProperties); + + //INVARIANT: TableScan with key (A) and TopN and/or Limit sets result N=1. Key property should be emptied. + expectedLogicalProperties = new LogicalPropertiesImpl( + new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(1L), + new KeyProperty()); + + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode customerTableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + + return p.topN(1, ImmutableList.of(customerCustKeyVariable), customerTableScan); + }) + .matches(expectedLogicalProperties); + } + + @Test + void testSortNodeLogicalProperties() + { + // Test KeyProperty propagation through sort. + LogicalProperties expectedLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty(ImmutableSet.of(new Key(ImmutableSet.of(customerCustKeyVariable))))); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode tableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + return p.sort(ImmutableList.of(customerCustKeyVariable), tableScan); + }) + .matches(expectedLogicalProperties); + + //TEST: Propagate maxcard through the filter below the sort + ConstantExpression constExpr = new ConstantExpression(100L, BIGINT); + EquivalenceClassProperty equivalenceClasses = new EquivalenceClassProperty(functionResolution); + equivalenceClasses.update(customerCustKeyVariable, constExpr); + + expectedLogicalProperties = new LogicalPropertiesImpl(equivalenceClasses, + new MaxCardProperty(1L), + new KeyProperty()); + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + TableScanNode tableScan = p.tableScan( + customerTableHandle, + ImmutableList.of(customerCustKeyVariable), + ImmutableMap.of(customerCustKeyVariable, customerCustKeyColumn), + TupleDomain.none(), + TupleDomain.none(), + tester().getTableConstraints(customerTableHandle)); + p.variable(customerCustKeyVariable); + FilterNode filterNode = p.filter(p.rowExpression("c_custkey = BIGINT '100'"), tableScan); + return p.sort(ImmutableList.of(customerCustKeyVariable), filterNode); + }) + .matches(expectedLogicalProperties); + } + + @Test + public void testDefaultLogicalProperties() + { + LogicalProperties defaultLogicalProperties = new LogicalPropertiesImpl(new EquivalenceClassProperty(functionResolution), + new MaxCardProperty(), + new KeyProperty()); + + //Union node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.union( + ImmutableListMultimap.builder() + .putAll(p.variable("o1"), p.variable("s1_c1"), p.variable("s2_c1")) + .putAll(p.variable("o2"), p.variable("s1_c2"), p.variable("s2_c2")) + .build(), + ImmutableList.of( + p.values(1, p.variable("s1_c1"), p.variable("s1_c2")), + p.values(2, p.variable("s2_c1"), p.variable("s2_c2"))))) + .matches(defaultLogicalProperties); + + //Intersect node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.intersect( + ImmutableListMultimap.builder() + .putAll(p.variable("o1"), p.variable("s1_c1"), p.variable("s2_c1")) + .putAll(p.variable("o2"), p.variable("s1_c2"), p.variable("s2_c2")) + .build(), + ImmutableList.of( + p.values(1, p.variable("s1_c1"), p.variable("s1_c2")), + p.values(2, p.variable("s2_c1"), p.variable("s2_c2"))))) + .matches(defaultLogicalProperties); + + //Lateral node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> p.lateral( + ImmutableList.of(p.variable("a")), + p.values(p.variable("a")), + p.values(p.variable("a")))) + .matches(defaultLogicalProperties); + + //MarkDistinct node should not propagate any logical properties + tester().assertThat(new NoOpRule(), logicalPropertiesProvider) + .on(p -> { + VariableReferenceExpression key = p.variable("key"); + VariableReferenceExpression mark = p.variable("mark"); + return p.markDistinct(mark, ImmutableList.of(key), p.values(10, key)); + }) + .matches(defaultLogicalProperties); + } + + private static class NoOpRule + implements Rule + { + private final Pattern pattern = Pattern.any(); + + @Override + public Pattern getPattern() + { + return pattern; + } + + @Override + public Result apply(PlanNode node, Captures captures, Context context) + { + return Result.ofPlanNode(node); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("pattern", pattern) + .toString(); + } + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java index 95ba4df4d45bb..536b0c2c06da2 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/BaseRuleTest.java @@ -26,7 +26,7 @@ public abstract class BaseRuleTest { - private RuleTester tester; + protected RuleTester tester; private List plugins; public BaseRuleTest(Plugin... plugins) @@ -35,7 +35,7 @@ public BaseRuleTest(Plugin... plugins) } @BeforeClass - public final void setUp() + public void setUp() { tester = new RuleTester(plugins); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java index 9c7b995acea7b..04e6470f37e3f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/PlanBuilder.java @@ -25,11 +25,13 @@ import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableHandle; import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.constraints.TableConstraint; import com.facebook.presto.spi.function.FunctionHandle; import com.facebook.presto.spi.plan.AggregationNode; import com.facebook.presto.spi.plan.AggregationNode.Aggregation; import com.facebook.presto.spi.plan.AggregationNode.Step; import com.facebook.presto.spi.plan.Assignments; +import com.facebook.presto.spi.plan.DistinctLimitNode; import com.facebook.presto.spi.plan.ExceptNode; import com.facebook.presto.spi.plan.FilterNode; import com.facebook.presto.spi.plan.IntersectNode; @@ -100,6 +102,7 @@ import java.util.function.Consumer; import java.util.stream.Stream; +import static com.facebook.presto.common.block.SortOrder.ASC_NULLS_FIRST; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.UnknownType.UNKNOWN; import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; @@ -269,10 +272,22 @@ public TopNNode topN(long count, List orderBy, Plan idAllocator.getNextId(), source, count, - new OrderingScheme(orderBy.stream().map(variable -> new Ordering(variable, SortOrder.ASC_NULLS_FIRST)).collect(toImmutableList())), + new OrderingScheme(orderBy.stream().map(variable -> new Ordering(variable, ASC_NULLS_FIRST)).collect(toImmutableList())), TopNNode.Step.SINGLE); } + public DistinctLimitNode distinctLimit(long count, List distinctSymbols, PlanNode source) + { + return new DistinctLimitNode( + source.getSourceLocation(), + idAllocator.getNextId(), + source, + count, + false, + distinctSymbols, + Optional.empty()); + } + public SampleNode sample(double sampleRatio, SampleNode.Type type, PlanNode source) { return new SampleNode(source.getSourceLocation(), idAllocator.getNextId(), source, sampleRatio, type); @@ -515,6 +530,26 @@ public TableScanNode tableScan( tableHandle, variables, assignments, + ImmutableList.of(), + currentConstraint, + enforcedConstraint); + } + + public TableScanNode tableScan( + TableHandle tableHandle, + List variables, + Map assignments, + TupleDomain currentConstraint, + TupleDomain enforcedConstraint, + List> tableConstraints) + { + return new TableScanNode( + Optional.empty(), + idAllocator.getNextId(), + tableHandle, + variables, + assignments, + tableConstraints, currentConstraint, enforcedConstraint); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java index f303ce8506309..029876dfa6a98 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleAssert.java @@ -26,6 +26,8 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.security.AccessControl; import com.facebook.presto.spi.WarningCollector; +import com.facebook.presto.spi.plan.LogicalProperties; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.spi.plan.PlanNode; import com.facebook.presto.spi.plan.PlanNodeId; import com.facebook.presto.spi.plan.PlanNodeIdAllocator; @@ -40,6 +42,7 @@ import com.facebook.presto.sql.planner.iterative.Memo; import com.facebook.presto.sql.planner.iterative.PlanNodeMatcher; import com.facebook.presto.sql.planner.iterative.Rule; +import com.facebook.presto.sql.planner.iterative.properties.LogicalPropertiesImpl; import com.facebook.presto.sql.planner.iterative.rule.TranslateExpressions; import com.facebook.presto.transaction.TransactionManager; import com.google.common.collect.ImmutableSet; @@ -70,8 +73,15 @@ public class RuleAssert private Session session; private TypeProvider types; private PlanNode plan; + private Optional logicalPropertiesProvider; public RuleAssert(Metadata metadata, StatsCalculator statsCalculator, CostCalculator costCalculator, Session session, Rule rule, TransactionManager transactionManager, AccessControl accessControl) + { + this(metadata, statsCalculator, costCalculator, session, rule, transactionManager, accessControl, Optional.empty()); + } + + public RuleAssert(Metadata metadata, StatsCalculator statsCalculator, CostCalculator costCalculator, Session session, Rule rule, + TransactionManager transactionManager, AccessControl accessControl, Optional logicalPropertiesProvider) { this.metadata = requireNonNull(metadata, "metadata is null"); this.statsCalculator = new TestingStatsCalculator(requireNonNull(statsCalculator, "statsCalculator is null")); @@ -80,6 +90,7 @@ public RuleAssert(Metadata metadata, StatsCalculator statsCalculator, CostCalcul this.rule = requireNonNull(rule, "rule is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); + this.logicalPropertiesProvider = requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider is null"); } public RuleAssert setSystemProperty(String key, String value) @@ -175,18 +186,42 @@ public void matches(PlanMatchPattern pattern) }); } + public void matches(LogicalProperties expectedLogicalProperties) + { + RuleApplication ruleApplication = applyRule(); + TypeProvider types = ruleApplication.types; + + if (!ruleApplication.wasRuleApplied()) { + fail(String.format( + "%s did not fire for:\n%s", + rule.getClass().getName(), + formatPlan(plan, types))); + } + + // ensure that the logical properties of the root group are equivalent to the expected logical properties + LogicalProperties rootNodeLogicalProperties = ruleApplication.getMemo().getLogicalProperties(ruleApplication.getMemo().getRootGroup()).get(); + if (!((LogicalPropertiesImpl) rootNodeLogicalProperties).equals((LogicalPropertiesImpl) expectedLogicalProperties)) { + fail(String.format( + "Logical properties of root node doesn't match expected logical properties\n" + + "\texpected: %s\n" + + "\tactual: %s", + expectedLogicalProperties, + rootNodeLogicalProperties)); + } + } + private RuleApplication applyRule() { PlanVariableAllocator variableAllocator = new PlanVariableAllocator(types.allVariables()); - Memo memo = new Memo(idAllocator, plan); + Memo memo = new Memo(idAllocator, plan, logicalPropertiesProvider); Lookup lookup = Lookup.from(planNode -> Stream.of(memo.resolve(planNode))); PlanNode memoRoot = memo.getNode(memo.getRootGroup()); - return inTransaction(session -> applyRule(rule, memoRoot, ruleContext(statsCalculator, costCalculator, variableAllocator, memo, lookup, session))); + return inTransaction(session -> applyRule(rule, memoRoot, ruleContext(statsCalculator, costCalculator, variableAllocator, memo, lookup, session), memo)); } - private static RuleApplication applyRule(Rule rule, PlanNode planNode, Rule.Context context) + private static RuleApplication applyRule(Rule rule, PlanNode planNode, Rule.Context context, Memo memo) { PlanNodeMatcher matcher = new PlanNodeMatcher(context.getLookup()); Match match = matcher.match(rule.getPattern(), planNode); @@ -199,7 +234,7 @@ private static RuleApplication applyRule(Rule rule, PlanNode planNode, Ru result = rule.apply(match.value(), match.captures(), context); } - return new RuleApplication(context.getLookup(), context.getStatsProvider(), context.getVariableAllocator().getTypes(), result); + return new RuleApplication(context.getLookup(), context.getStatsProvider(), context.getVariableAllocator().getTypes(), memo, result); } private String formatPlan(PlanNode plan, TypeProvider types) @@ -286,13 +321,15 @@ private static class RuleApplication private final StatsProvider statsProvider; private final TypeProvider types; private final Rule.Result result; + private final Memo memo; - public RuleApplication(Lookup lookup, StatsProvider statsProvider, TypeProvider types, Rule.Result result) + public RuleApplication(Lookup lookup, StatsProvider statsProvider, TypeProvider types, Memo memo, Rule.Result result) { this.lookup = requireNonNull(lookup, "lookup is null"); this.statsProvider = requireNonNull(statsProvider, "statsProvider is null"); this.types = requireNonNull(types, "types is null"); this.result = requireNonNull(result, "result is null"); + this.memo = requireNonNull(memo, "memo is null"); } private boolean wasRuleApplied() @@ -304,6 +341,11 @@ public PlanNode getTransformedPlan() { return result.getTransformedPlan().orElseThrow(() -> new IllegalStateException("Rule did not produce transformed plan")); } + + private Memo getMemo() + { + return memo; + } } public static class TestingStatsCalculator diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java index c441a6c471b9b..780ab3885d73e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/iterative/rule/test/RuleTester.java @@ -16,8 +16,13 @@ import com.facebook.presto.Session; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.security.AccessControl; +import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.TableHandle; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.spi.constraints.TableConstraint; +import com.facebook.presto.spi.plan.LogicalPropertiesProvider; import com.facebook.presto.split.PageSourceManager; import com.facebook.presto.split.SplitManager; import com.facebook.presto.sql.parser.SqlParser; @@ -68,6 +73,11 @@ public RuleTester(List plugins, Map sessionProperties) } public RuleTester(List plugins, Map sessionProperties, Optional nodeCountForStats) + { + this(plugins, sessionProperties, nodeCountForStats, new TpchConnectorFactory(1)); + } + + public RuleTester(List plugins, Map sessionProperties, Optional nodeCountForStats, ConnectorFactory connectorFactory) { Session.SessionBuilder sessionBuilder = testSessionBuilder() .setCatalog(CATALOG_ID) @@ -84,7 +94,7 @@ public RuleTester(List plugins, Map sessionProperties, O .map(nodeCount -> LocalQueryRunner.queryRunnerWithFakeNodeCountForStats(session, nodeCount)) .orElseGet(() -> new LocalQueryRunner(session)); queryRunner.createCatalog(session.getCatalog().get(), - new TpchConnectorFactory(1), + connectorFactory, ImmutableMap.of()); plugins.stream().forEach(queryRunner::installPlugin); @@ -101,6 +111,11 @@ public RuleAssert assertThat(Rule rule) return new RuleAssert(metadata, queryRunner.getStatsCalculator(), queryRunner.getEstimatedExchangesCostCalculator(), session, rule, transactionManager, accessControl); } + public RuleAssert assertThat(Rule rule, LogicalPropertiesProvider logicalPropertiesProvider) + { + return new RuleAssert(metadata, queryRunner.getStatsCalculator(), queryRunner.getEstimatedExchangesCostCalculator(), session, rule, transactionManager, accessControl, Optional.of(logicalPropertiesProvider)); + } + public OptimizerAssert assertThat(PlanOptimizer optimizer) { return new OptimizerAssert(metadata, queryRunner.getStatsCalculator(), session, optimizer, transactionManager, accessControl); @@ -136,4 +151,12 @@ public ConnectorId getCurrentConnectorId() { return queryRunner.inTransaction(transactionSession -> metadata.getCatalogHandle(transactionSession, session.getCatalog().get())).get(); } + + public List> getTableConstraints(TableHandle tableHandle) + { + return queryRunner.inTransaction(transactionSession -> { + metadata.getCatalogHandle(transactionSession, session.getCatalog().get()); + return metadata.getTableMetadata(transactionSession, tableHandle).getMetadata().getTableConstraints(); + }); + } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java index fbaedeb616d08..17a86047b06b2 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/AggregationNode.java @@ -96,6 +96,16 @@ public AggregationNode( this.outputs = unmodifiableList(outputs); } + /** + * Whether this node corresponds to a DISTINCT operation in SQL + */ + public static boolean isDistinct(AggregationNode node) + { + return node.getAggregations().isEmpty() && + node.getOutputVariables().size() == node.getGroupingKeys().size() && + node.getOutputVariables().containsAll(node.getGroupingKeys()); + } + public List getGroupingKeys() { return groupingSets.getGroupingKeys(); @@ -136,6 +146,13 @@ public List getSources() return unmodifiableList(Collections.singletonList(source)); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getAggregationProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java index bd5a3e1abf1aa..0f8b16635faa4 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/DistinctLimitNode.java @@ -94,6 +94,13 @@ public List getDistinctVariables() return distinctVariables; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getDistinctLimitProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java index 4dcd4253d589a..0175a2f774f5b 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/FilterNode.java @@ -27,6 +27,7 @@ import static java.util.Collections.singletonList; import static java.util.Collections.unmodifiableList; +import static java.util.Objects.requireNonNull; @Immutable public final class FilterNode @@ -95,6 +96,13 @@ public PlanNode replaceChildren(List newChildren) return new FilterNode(getSourceLocation(), getId(), newChildren.get(0), predicate); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getFilterProperties(this); + } + @Override public boolean equals(Object o) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java index 0b6cd1f552d35..6c0aea79b1ec1 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java @@ -101,6 +101,13 @@ public boolean isPartial() return step == Step.PARTIAL; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getLimitProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.java new file mode 100644 index 0000000000000..a7e060c7c6ede --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalProperties.java @@ -0,0 +1,56 @@ +/* + * 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.spi.plan; + +import com.facebook.presto.spi.relation.VariableReferenceExpression; + +import java.util.Set; + +/** + * Logical properties represent constraints that hold for a final or intermediate result produced by a PlanNode. + * They are used by the optimizer to enable query transformations that lead to better performing plans. + * For example, unique constraints can be used to eliminate redundant distinct operations. + * This is the interface used by optimizer rules to perform optimizations based upon logical properties. + *

+ * In this initial phase logical properties computed are based solely upon key constraints. In the future + * support for referential constraints, functional dependencies, and others will be added and + * hence this interface will become more robust over time. + */ +public interface LogicalProperties +{ + /** + * Determines if the set of variables form a unique constraint for a final or + * intermediate result produced by a PlanNode. + * + * @param keyVars + * @return True if the set of variables form a unique constraint or false otherwise. + */ + boolean isDistinct(Set keyVars); + + /** + * Determines if there is provably at most one tuple in a final or + * intermediate result set produced by a PlanNode. + * + * @return True if there is provably at most one tuple or false otherwise. + */ + boolean isAtMostSingleRow(); + + /** + * Determines if there is provably at most n tuples in a final or + * intermediate result set produced by a PlanNode. + * + * @return True if there is provably at most one tuple or false otherwise. + */ + boolean isAtMost(long n); +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java new file mode 100644 index 0000000000000..700c756363892 --- /dev/null +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/LogicalPropertiesProvider.java @@ -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.spi.plan; + +/** + * Defines a suite of plan node-specific methods for the computation of logical properties. + * Supplies a default implementation that produces an empty set of logical properties, and additionally, + * a suite of plan-node specific overrides of the default implementation. The implementations leverage + * property propagation builders supplied by LogicalPropertiesImpl. The LogicalPropertiesProvider + * mechanism enables a plan node to receive its logical property compute capabilities via dependency injection. + * This is needed because the computation of logical properties requires analysis of plan node's argument expressions, + * and the code the performs this analysis must be built in presto-main as this is where expression related classes are visible. + * The property computation implementation is dynamically injected into the presto-spi and presto-main plan node method's responsible + * for computing logical properties. This interface is defined in presto-spi where it is visible to all plan nodes. The + * implementation LogicalPropertiesImpl is provided in presto-main as per the reasons described above. + */ +public interface LogicalPropertiesProvider +{ + LogicalProperties getDefaultProperties(); + + LogicalProperties getTableScanProperties(TableScanNode tableScanNode); + + LogicalProperties getProjectProperties(ProjectNode projectNode); + + LogicalProperties getFilterProperties(FilterNode filterNode); + + LogicalProperties getJoinProperties(PlanNode joinNode); + + LogicalProperties getSemiJoinProperties(PlanNode semiJNode); + + LogicalProperties getSortProperties(PlanNode sortNode); + + LogicalProperties getAggregationProperties(AggregationNode aggregationNode); + + LogicalProperties getLimitProperties(LimitNode limitNode); + + LogicalProperties getTopNProperties(TopNNode limitNode); + + LogicalProperties getValuesProperties(ValuesNode valuesNode); + + LogicalProperties getDistinctLimitProperties(DistinctLimitNode limitNode); + + LogicalProperties getAssignUniqueIdProperties(PlanNode assignUniqueId); +} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java index aca3db7fbc395..ee9d036ac604e 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/PlanNode.java @@ -57,6 +57,15 @@ public Optional getSourceLocation() */ public abstract List getSources(); + /** + * Logical properties are a function of source properties and the operation performed by the plan node + */ + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getDefaultProperties(); + } + /** * The output from the upstream PlanNodes. * It should serve as the input for the current PlanNode. diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java index a25530d5653d2..20a476f5dd026 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ProjectNode.java @@ -107,6 +107,13 @@ public PlanNode replaceChildren(List newChildren) return new ProjectNode(getSourceLocation(), getId(), newChildren.get(0), assignments, locality); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getProjectProperties(this); + } + @Override public boolean equals(Object o) { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java index bed136ac1c9ad..24037ccaa3cd9 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TableScanNode.java @@ -169,6 +169,13 @@ public List getSources() return emptyList(); } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getTableScanProperties(this); + } + @Override @JsonProperty public List getOutputVariables() diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java index c5b1f7baad689..267b600d7fca3 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/TopNNode.java @@ -88,6 +88,13 @@ public PlanNode getSource() return source; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getTopNProperties(this); + } + @Override public List getOutputVariables() { diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java index e22f45d609c92..1f5f25a226565 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/plan/ValuesNode.java @@ -62,6 +62,13 @@ public List> getRows() return rows; } + @Override + public LogicalProperties computeLogicalProperties(LogicalPropertiesProvider logicalPropertiesProvider) + { + requireNonNull(logicalPropertiesProvider, "logicalPropertiesProvider cannot be null."); + return logicalPropertiesProvider.getValuesProperties(this); + } + @Override @JsonProperty public List getOutputVariables()