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()