[feature](Nereids): add functional dependencies (#27051)

Add functional dependencies: including

- uniform slot, which means ndv <= 0
- unique slot, which means ndv = row cound
This commit is contained in:
谢健
2023-11-29 12:46:25 +08:00
committed by GitHub
parent 2a66a5d167
commit 2b375f66e0
61 changed files with 1321 additions and 50 deletions

View File

@ -23,6 +23,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.UnboundLogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Sink;
@ -40,7 +41,8 @@ import java.util.Optional;
/**
* Represent an olap table sink plan node that has not been bound.
*/
public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE> implements Unbound, Sink {
public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE>
implements Unbound, Sink, BlockFuncDepsPropagation {
private final List<String> nameParts;
private final List<String> colNames;

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.properties.UnboundLogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
@ -41,7 +42,8 @@ import java.util.Optional;
* A relation that contains only one row consist of some constant expressions.
* e.g. select 100, 'value'
*/
public class UnboundOneRowRelation extends LogicalRelation implements Unbound, OneRowRelation {
public class UnboundOneRowRelation extends LogicalRelation implements Unbound, OneRowRelation,
BlockFuncDepsPropagation {
private final List<NamedExpression> projects;

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.properties.UnboundLogicalProperties;
import org.apache.doris.nereids.trees.TableSample;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
@ -43,7 +44,7 @@ import java.util.Optional;
/**
* Represent a relation plan node that has not been bound.
*/
public class UnboundRelation extends LogicalRelation implements Unbound {
public class UnboundRelation extends LogicalRelation implements Unbound, BlockFuncDepsPropagation {
private final List<String> nameParts;
private final List<String> partNames;

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Sink;
@ -38,7 +39,8 @@ import java.util.Optional;
/**
* unbound result sink
*/
public class UnboundResultSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE> implements Unbound, Sink {
public class UnboundResultSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE>
implements Unbound, Sink, BlockFuncDepsPropagation {
public UnboundResultSink(CHILD_TYPE child) {
super(PlanType.LOGICAL_UNBOUND_RESULT_SINK, ImmutableList.of(), child);

View File

@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Properties;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
@ -38,7 +39,8 @@ import java.util.Objects;
import java.util.Optional;
/** UnboundTVFRelation */
public class UnboundTVFRelation extends LogicalRelation implements TVFRelation, Unbound {
public class UnboundTVFRelation extends LogicalRelation implements TVFRelation, Unbound,
BlockFuncDepsPropagation {
private final String functionName;
private final Properties properties;

View File

@ -27,6 +27,7 @@ import org.apache.doris.nereids.memo.CopyInResult;
import org.apache.doris.nereids.memo.Group;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.memo.Memo;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.PhysicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
@ -214,7 +215,7 @@ public class PlanReceiver implements AbstractReceiver {
List<Expression> otherConjuncts) {
// Check whether only NSL can be performed
LogicalProperties joinProperties = new LogicalProperties(
() -> JoinUtils.getJoinOutput(joinType, left, right));
() -> JoinUtils.getJoinOutput(joinType, left, right), () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
List<Plan> plans = Lists.newArrayList();
if (JoinUtils.shouldNestedLoopJoin(joinType, hashConjuncts)) {
plans.add(new PhysicalNestedLoopJoin<>(joinType, hashConjuncts, otherConjuncts,
@ -378,7 +379,9 @@ public class PlanReceiver implements AbstractReceiver {
.collect(Collectors.toList());
if (!outputSet.equals(new HashSet<>(projects))) {
LogicalProperties projectProperties = new LogicalProperties(
() -> projects.stream().map(p -> p.toSlot()).collect(Collectors.toList()));
() -> projects.stream()
.map(p -> p.toSlot())
.collect(Collectors.toList()), () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
allChild = allChild.stream()
.map(c -> new PhysicalProject<>(projects, projectProperties, c))
.collect(Collectors.toList());

View File

@ -0,0 +1,232 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.properties;
import org.apache.doris.nereids.trees.expressions.Slot;
import com.google.common.collect.ImmutableSet;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Record functional dependencies, aka func deps, including
* 1. unique slot: it means the column that has ndv = row count, can be null
* 2. uniform slotL it means the column that has ndv = 1, can be null
*/
public class FunctionalDependencies {
public static final FunctionalDependencies EMPTY_FUNC_DEPS
= new FunctionalDependencies(new NestedSet().toImmutable(), new NestedSet().toImmutable());
private final NestedSet uniqueSet;
private final NestedSet uniformSet;
private FunctionalDependencies(NestedSet uniqueSet, NestedSet uniformSet) {
this.uniqueSet = uniqueSet;
this.uniformSet = uniformSet;
}
public boolean isEmpty() {
return uniformSet.isEmpty() && uniqueSet.isEmpty();
}
public boolean isUnique(Slot slot) {
return uniqueSet.contains(slot);
}
public boolean isUnique(Set<Slot> slotSet) {
return !slotSet.isEmpty() && uniqueSet.containsAnySub(slotSet);
}
public boolean isUniform(Slot slot) {
return uniformSet.contains(slot);
}
public boolean isUniform(ImmutableSet<Slot> slotSet) {
return !slotSet.isEmpty()
&& (uniformSet.contains(slotSet) || slotSet.stream().allMatch(uniformSet::contains));
}
public boolean isUniqueAndNotNull(Slot slot) {
return !slot.nullable() && isUnique(slot);
}
public boolean isUniqueAndNotNull(Set<Slot> slotSet) {
return slotSet.stream().noneMatch(Slot::nullable) && isUnique(slotSet);
}
public boolean isUniformAndNotNull(Slot slot) {
return !slot.nullable() && isUniform(slot);
}
public boolean isUniformAndNotNull(ImmutableSet<Slot> slotSet) {
return slotSet.stream().noneMatch(Slot::nullable) && isUniform(slotSet);
}
@Override
public String toString() {
return "uniform:" + uniformSet
+ "\t unique:" + uniqueSet;
}
/**
* Builder of Func Deps
*/
public static class Builder {
private final NestedSet uniqueSet;
private final NestedSet uniformSet;
public Builder() {
uniqueSet = new NestedSet();
uniformSet = new NestedSet();
}
public Builder(FunctionalDependencies other) {
this.uniformSet = new NestedSet(other.uniformSet);
this.uniqueSet = new NestedSet(other.uniqueSet);
}
public void addUniformSlot(Slot slot) {
uniformSet.add(slot);
}
public void addUniformSlot(FunctionalDependencies functionalDependencies) {
uniformSet.add(functionalDependencies.uniformSet);
}
public void addUniformSlot(ImmutableSet<Slot> slotSet) {
uniformSet.add(slotSet);
}
public void addUniqueSlot(Slot slot) {
uniqueSet.add(slot);
}
public void addUniqueSlot(ImmutableSet<Slot> slotSet) {
uniqueSet.add(slotSet);
}
public void addFunctionalDependencies(FunctionalDependencies fd) {
uniformSet.add(fd.uniformSet);
uniqueSet.add(fd.uniqueSet);
}
public FunctionalDependencies build() {
return new FunctionalDependencies(uniqueSet.toImmutable(), uniformSet.toImmutable());
}
public void pruneSlots(Set<Slot> outputSlots) {
uniformSet.removeNotContain(outputSlots);
uniqueSet.removeNotContain(outputSlots);
}
public void replace(Map<Slot, Slot> replaceMap) {
uniformSet.replace(replaceMap);
uniqueSet.replace(replaceMap);
}
}
static class NestedSet {
Set<Slot> slots;
Set<ImmutableSet<Slot>> slotSets;
NestedSet() {
slots = new HashSet<>();
slotSets = new HashSet<>();
}
NestedSet(NestedSet o) {
this.slots = new HashSet<>(o.slots);
this.slotSets = new HashSet<>(o.slotSets);
}
NestedSet(Set<Slot> slots, Set<ImmutableSet<Slot>> slotSets) {
this.slots = slots;
this.slotSets = slotSets;
}
public boolean contains(Slot slot) {
return slots.contains(slot);
}
public boolean contains(ImmutableSet<Slot> slotSet) {
if (slotSet.size() == 1) {
return slots.contains(slotSet.iterator().next());
}
return slotSets.contains(slotSet);
}
public boolean containsAnySub(Set<Slot> slotSet) {
return slotSet.stream().anyMatch(s -> slots.contains(s))
|| slotSets.stream().anyMatch(slotSet::containsAll);
}
public void removeNotContain(Set<Slot> slotSet) {
slots = slots.stream()
.filter(slotSet::contains)
.collect(Collectors.toSet());
slotSets = slotSets.stream()
.filter(slotSet::containsAll)
.collect(Collectors.toSet());
}
public void add(Slot slot) {
slots.add(slot);
}
public void add(ImmutableSet<Slot> slotSet) {
if (slotSet.isEmpty()) {
return;
}
if (slotSet.size() == 1) {
slots.add(slotSet.iterator().next());
return;
}
slotSets.add(slotSet);
}
public void add(NestedSet nestedSet) {
slots.addAll(nestedSet.slots);
slotSets.addAll(nestedSet.slotSets);
}
public boolean isEmpty() {
return slots.isEmpty() && slotSets.isEmpty();
}
@Override
public String toString() {
return "{" + slots + slotSets + "}";
}
public void replace(Map<Slot, Slot> replaceMap) {
slots = slots.stream()
.map(s -> replaceMap.getOrDefault(s, s))
.collect(Collectors.toSet());
slotSets = slotSets.stream()
.map(set -> set.stream().map(replaceMap::get).collect(ImmutableSet.toImmutableSet()))
.collect(Collectors.toSet());
}
public NestedSet toImmutable() {
return new NestedSet(ImmutableSet.copyOf(slots), ImmutableSet.copyOf(slotSets));
}
}
}

View File

@ -42,10 +42,11 @@ public class LogicalProperties {
protected final Supplier<Set<Slot>> outputSetSupplier;
protected final Supplier<Map<Slot, Slot>> outputMapSupplier;
protected final Supplier<Set<ExprId>> outputExprIdSetSupplier;
protected final Supplier<FunctionalDependencies> fdSupplier;
private Integer hashCode = null;
public LogicalProperties(Supplier<List<Slot>> outputSupplier) {
this(outputSupplier, ImmutableList::of);
public LogicalProperties(Supplier<List<Slot>> outputSupplier, Supplier<FunctionalDependencies> fdSupplier) {
this(outputSupplier, fdSupplier, ImmutableList::of);
}
/**
@ -54,7 +55,8 @@ public class LogicalProperties {
* @param outputSupplier provide the output. Supplier can lazy compute output without
* throw exception for which children have UnboundRelation
*/
public LogicalProperties(Supplier<List<Slot>> outputSupplier, Supplier<List<Slot>> nonUserVisibleOutputSupplier) {
public LogicalProperties(Supplier<List<Slot>> outputSupplier,
Supplier<FunctionalDependencies> fdSupplier, Supplier<List<Slot>> nonUserVisibleOutputSupplier) {
this.outputSupplier = Suppliers.memoize(
Objects.requireNonNull(outputSupplier, "outputSupplier can not be null")
);
@ -73,6 +75,9 @@ public class LogicalProperties {
.map(NamedExpression::getExprId)
.collect(ImmutableSet.toImmutableSet())
);
this.fdSupplier = Suppliers.memoize(
Objects.requireNonNull(fdSupplier, "FunctionalDependencies can not be null")
);
}
public List<Slot> getOutput() {
@ -91,6 +96,10 @@ public class LogicalProperties {
return outputExprIdSetSupplier.get();
}
public FunctionalDependencies getFunctionalDependencies() {
return fdSupplier.get();
}
public List<Id> getOutputExprIds() {
return outputExprIdsSupplier.get();
}

View File

@ -32,7 +32,7 @@ public class UnboundLogicalProperties extends LogicalProperties {
public static final UnboundLogicalProperties INSTANCE = new UnboundLogicalProperties();
private UnboundLogicalProperties() {
super(ImmutableList::of);
super(ImmutableList::of, () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
}
@Override

View File

@ -21,6 +21,7 @@ import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
@ -52,7 +53,7 @@ import java.util.Optional;
* firstChild [RIGHT (OUTER/SEMI/ANTI) JOIN] children[1, last].
* eg: MJ([ROJ] A, B, C, D) is {A} [ROJ] {B C D}.
*/
public class MultiJoin extends AbstractLogicalPlan {
public class MultiJoin extends AbstractLogicalPlan implements BlockFuncDepsPropagation {
/*
* topJoin
* / \ MultiJoin

View File

@ -19,12 +19,14 @@ package org.apache.doris.nereids.trees.plans;
import org.apache.doris.nereids.analyzer.Unbound;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.UnboundLogicalProperties;
import org.apache.doris.nereids.trees.AbstractTreeNode;
import org.apache.doris.nereids.trees.expressions.ExprId;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.util.MutableState;
import org.apache.doris.nereids.util.MutableState.EmptyMutableState;
import org.apache.doris.nereids.util.TreeStringUtils;
@ -169,7 +171,11 @@ public abstract class AbstractPlan extends AbstractTreeNode<Plan> implements Pla
if (hasUnboundChild || hasUnboundExpression()) {
return UnboundLogicalProperties.INSTANCE;
} else {
return new LogicalProperties(this::computeOutput);
Supplier<List<Slot>> outputSupplier = Suppliers.memoize(this::computeOutput);
Supplier<FunctionalDependencies> fdSupplier = () -> this instanceof LogicalPlan
? ((LogicalPlan) this).computeFuncDeps(outputSupplier)
: FunctionalDependencies.EMPTY_FUNC_DEPS;
return new LogicalProperties(outputSupplier, fdSupplier);
}
}

View File

@ -0,0 +1,35 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.plans;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import java.util.List;
import java.util.function.Supplier;
/**
* Block fd propagation, it always returns an empty fd
*/
public interface BlockFuncDepsPropagation extends LogicalPlan {
@Override
default FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
return FunctionalDependencies.EMPTY_FUNC_DEPS;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.plans;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -77,7 +78,7 @@ public class FakePlan implements Plan {
@Override
public LogicalProperties getLogicalProperties() {
return new LogicalProperties(ArrayList::new);
return new LogicalProperties(ArrayList::new, () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
}
@Override

View File

@ -37,7 +37,7 @@ import java.util.Optional;
* Used in {@link org.apache.doris.nereids.pattern.GroupExpressionMatching.GroupExpressionIterator},
* as a place-holder when do match root.
*/
public class GroupPlan extends LogicalLeaf {
public class GroupPlan extends LogicalLeaf implements BlockFuncDepsPropagation {
private final Group group;

View File

@ -160,6 +160,10 @@ public enum JoinType {
return this == LEFT_ANTI_JOIN;
}
public final boolean isLefSemiJoin() {
return this == LEFT_SEMI_JOIN;
}
public final boolean isRightSemiOrAntiJoin() {
return this == RIGHT_SEMI_JOIN || this == RIGHT_ANTI_JOIN;
}
@ -168,6 +172,10 @@ public enum JoinType {
return this == RIGHT_ANTI_JOIN;
}
public final boolean isRightSemiJoin() {
return this == RIGHT_SEMI_JOIN;
}
public final boolean isSemiOrAntiJoin() {
return this == LEFT_SEMI_JOIN || this == RIGHT_SEMI_JOIN || this == LEFT_ANTI_JOIN
|| this == NULL_AWARE_LEFT_ANTI_JOIN || this == RIGHT_ANTI_JOIN;

View File

@ -0,0 +1,44 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.plans;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import java.util.List;
import java.util.function.Supplier;
/**
* Propagate fd, keep children's fd
*/
public interface PropagateFuncDeps extends LogicalPlan {
@Override
default FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
if (children().size() == 1) {
// Note when changing function dependencies, we always clone it.
// So it's safe to return a reference
return child(0).getLogicalProperties().getFunctionalDependencies();
}
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder();
children().stream()
.map(p -> p.getLogicalProperties().getFunctionalDependencies())
.forEach(builder::addFunctionalDependencies);
return builder.build();
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.AbstractPlan;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
@ -36,7 +37,7 @@ import java.util.Optional;
/**
* All DDL and DML commands' super class.
*/
public abstract class Command extends AbstractPlan implements LogicalPlan {
public abstract class Command extends AbstractPlan implements LogicalPlan, BlockFuncDepsPropagation {
protected Command(PlanType type) {
super(type, Optional.empty(), Optional.empty(), null, ImmutableList.of());

View File

@ -18,23 +18,36 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.VirtualSlotReference;
import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait;
import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
import org.apache.doris.nereids.trees.expressions.functions.agg.Ndv;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Aggregate;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.ExpressionUtils;
import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import java.util.stream.Collectors;
/**
@ -282,4 +295,72 @@ public class LogicalAggregate<CHILD_TYPE extends Plan>
return new LogicalAggregate<>(normalizedGroupBy, normalizedOutput, true, ordinalIsResolved, generated,
hasPushed, sourceRepeat, Optional.empty(), Optional.empty(), normalizedChild);
}
private void updateFuncDepsGroupByUnique(NamedExpression namedExpression, Builder fdBuilder) {
if (ExpressionUtils.isInjective(namedExpression)) {
fdBuilder.addUniqueSlot(ImmutableSet.copyOf(namedExpression.getInputSlots()));
return;
}
if (!(namedExpression instanceof Alias && namedExpression.child(0) instanceof AggregateFunction)) {
return;
}
AggregateFunction agg = (AggregateFunction) namedExpression.child(0);
if (agg instanceof Count || agg instanceof Ndv) {
fdBuilder.addUniformSlot(namedExpression.toSlot());
return;
}
if (ExpressionUtils.isInjectiveAgg(agg)
&& child().getLogicalProperties().getFunctionalDependencies().isUniqueAndNotNull(agg.getInputSlots())) {
fdBuilder.addUniqueSlot(namedExpression.toSlot());
}
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies childFd = child(0).getLogicalProperties().getFunctionalDependencies();
Set<Slot> outputSet = new HashSet<>(outputSupplier.get());
Builder fdBuilder = new Builder();
// when group by all tuples, the result only have one row
if (groupByExpressions.isEmpty()) {
outputSet.forEach(s -> {
fdBuilder.addUniformSlot(s);
fdBuilder.addUniqueSlot(s);
});
return fdBuilder.build();
}
// when group by complicate expression or virtual slot, just propagate uniform slots
if (groupByExpressions.stream()
.anyMatch(s -> !(s instanceof SlotReference) || s instanceof VirtualSlotReference)) {
fdBuilder.addUniformSlot(childFd);
fdBuilder.pruneSlots(outputSet);
return fdBuilder.build();
}
// when group by uniform slot, the result only have one row
ImmutableSet<Slot> groupByKeys = groupByExpressions.stream()
.map(s -> (Slot) s)
.collect(ImmutableSet.toImmutableSet());
if (childFd.isUniformAndNotNull(groupByKeys)) {
outputSupplier.get().forEach(s -> {
fdBuilder.addUniformSlot(s);
fdBuilder.addUniqueSlot(s);
});
}
// when group by unique slot, the result depends on agg func
if (childFd.isUniqueAndNotNull(groupByKeys)) {
for (NamedExpression namedExpression : getOutputExpressions()) {
updateFuncDepsGroupByUnique(namedExpression, fdBuilder);
}
}
// group by keys is unique
fdBuilder.addUniqueSlot(groupByKeys);
fdBuilder.pruneSlots(outputSet);
return fdBuilder.build();
}
}

View File

@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -44,7 +45,7 @@ import java.util.Optional;
* @param <RIGHT_CHILD_TYPE> subquery.
*/
public class LogicalApply<LEFT_CHILD_TYPE extends Plan, RIGHT_CHILD_TYPE extends Plan>
extends LogicalBinary<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> {
extends LogicalBinary<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> implements PropagateFuncDeps {
// correlation column
private final List<Expression> correlationSlot;

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -40,7 +41,8 @@ import java.util.Optional;
* If the number of rows is more than the desired num of rows, the query will be cancelled.
* The cancelled reason will be reported by Backend and displayed back to the user.
*/
public class LogicalAssertNumRows<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> {
public class LogicalAssertNumRows<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements
PropagateFuncDeps {
private final AssertNumRowsElement assertNumRowsElement;

View File

@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -36,7 +37,7 @@ import java.util.Optional;
/**
* Logical Node for CTE
*/
public class LogicalCTE<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> {
public class LogicalCTE<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements PropagateFuncDeps {
private final List<LogicalSubQueryAlias<Plan>> aliasQueries;

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -36,7 +37,8 @@ import java.util.Optional;
* LogicalCTEAnchor
*/
public class LogicalCTEAnchor<LEFT_CHILD_TYPE extends Plan,
RIGHT_CHILD_TYPE extends Plan> extends LogicalBinary<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> {
RIGHT_CHILD_TYPE extends Plan> extends LogicalBinary<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> implements
PropagateFuncDeps {
private final CTEId cteId;

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.CTEId;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
@ -40,7 +41,8 @@ import java.util.Optional;
/**
* LogicalCTEConsumer
*/
public class LogicalCTEConsumer extends LogicalRelation {
//TODO: find cte producer and propagate its functional dependencies
public class LogicalCTEConsumer extends LogicalRelation implements BlockFuncDepsPropagation {
private final String name;
private final CTEId cteId;

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -37,7 +38,8 @@ import java.util.Optional;
/**
* LogicalCTEProducer
*/
public class LogicalCTEProducer<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> {
public class LogicalCTEProducer<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements
PropagateFuncDeps {
private final CTEId cteId;

View File

@ -19,10 +19,13 @@ package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.catalog.DatabaseIf;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.datasource.CatalogIf;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
@ -33,11 +36,13 @@ import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.commons.lang3.StringUtils;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* abstract class catalog relation for logical relation
@ -113,4 +118,18 @@ public abstract class LogicalCatalogRelation extends LogicalRelation implements
}
return Utils.qualifiedName(qualifier, table.getName());
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
Builder fdBuilder = new Builder();
if (table instanceof OlapTable && ((OlapTable) table).getKeysType().isAggregationFamily()) {
ImmutableSet<Slot> slotSet = computeOutput().stream()
.filter(SlotReference.class::isInstance)
.filter(s -> ((SlotReference) s).getColumn().isPresent()
&& ((SlotReference) s).getColumn().get().isKey())
.collect(ImmutableSet.toImmutableSet());
fdBuilder.addUniqueSlot(slotSet);
}
return fdBuilder.build();
}
}

View File

@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation;
import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
@ -46,7 +47,8 @@ import java.util.Optional;
/**
* Logical Check Policy
*/
public class LogicalCheckPolicy<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> {
public class LogicalCheckPolicy<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements
PropagateFuncDeps {
public LogicalCheckPolicy(CHILD_TYPE child) {
super(PlanType.LOGICAL_CHECK_POLICY, child);

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.algebra.Sink;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -37,7 +38,7 @@ import java.util.Optional;
* use for defer materialize top n
*/
public class LogicalDeferMaterializeResultSink<CHILD_TYPE extends Plan>
extends LogicalSink<CHILD_TYPE> implements Sink {
extends LogicalSink<CHILD_TYPE> implements Sink, PropagateFuncDeps {
private final LogicalResultSink<? extends Plan> logicalResultSink;
private final OlapTable olapTable;

View File

@ -18,6 +18,8 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.OrderKey;
import org.apache.doris.nereids.trees.expressions.ExprId;
@ -37,11 +39,13 @@ import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
/**
* use for defer materialize top n
*/
public class LogicalDeferMaterializeTopN<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements TopN {
public class LogicalDeferMaterializeTopN<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE>
implements TopN {
private final LogicalTopN<? extends Plan> logicalTopN;
@ -111,6 +115,19 @@ public class LogicalDeferMaterializeTopN<CHILD_TYPE extends Plan> extends Logica
.collect(ImmutableList.toImmutableList());
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies fd = child(0).getLogicalProperties().getFunctionalDependencies();
if (getLimit() == 1) {
Builder builder = new Builder();
List<Slot> output = outputSupplier.get();
output.forEach(builder::addUniformSlot);
output.forEach(builder::addUniqueSlot);
fd = builder.build();
}
return fd;
}
@Override
public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
return visitor.visitLogicalDeferMaterializeTopN(this, context);

View File

@ -21,6 +21,7 @@ import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
@ -39,7 +40,8 @@ import java.util.Optional;
* e.g.
* select * from tbl limit 0
*/
public class LogicalEmptyRelation extends LogicalRelation implements EmptyRelation, OutputPrunable {
public class LogicalEmptyRelation extends LogicalRelation
implements EmptyRelation, OutputPrunable, BlockFuncDepsPropagation {
private final List<NamedExpression> projects;

View File

@ -18,8 +18,10 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
@ -27,9 +29,13 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Supplier;
/**
* Logical Except.
@ -97,4 +103,23 @@ public class LogicalExcept extends LogicalSetOperation {
return new LogicalExcept(qualifier, newOutputs, regularChildrenOutputs,
Optional.empty(), Optional.empty(), children);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies
.Builder(child(0).getLogicalProperties().getFunctionalDependencies());
Map<Slot, Slot> replaceMap = new HashMap<>();
List<Slot> output = outputSupplier.get();
List<? extends Slot> originalOutputs = regularChildrenOutputs.isEmpty()
? child(0).getOutput()
: regularChildrenOutputs.get(0);
for (int i = 0; i < output.size(); i++) {
replaceMap.put(originalOutputs.get(i), output.get(i));
}
builder.replace(replaceMap);
if (qualifier == Qualifier.DISTINCT) {
builder.addUniqueSlot(ImmutableSet.copyOf(outputSupplier.get()));
}
return builder.build();
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.algebra.Sink;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
@ -36,7 +37,8 @@ import java.util.Optional;
/**
* logicalFileSink for select into outfile
*/
public class LogicalFileSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE> implements Sink {
public class LogicalFileSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE>
implements Sink, PropagateFuncDeps {
private final String filePath;
private final String format;

View File

@ -18,6 +18,8 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -26,6 +28,7 @@ import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Filter;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.ExpressionUtils;
import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
@ -37,6 +40,7 @@ import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@ -136,4 +140,12 @@ public class LogicalFilter<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_T
public LogicalFilter<Plan> withConjunctsAndChild(Set<Expression> conjuncts, Plan child) {
return new LogicalFilter<>(conjuncts, child);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
Builder fdBuilder = new Builder(
child().getLogicalProperties().getFunctionalDependencies());
getConjuncts().forEach(e -> fdBuilder.addUniformSlot(ExpressionUtils.extractUniformSlot(e)));
return fdBuilder.build();
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -35,6 +36,7 @@ import com.google.common.collect.Lists;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* plan for table generator, the statement like: SELECT * FROM tbl LATERAL VIEW EXPLODE(c1) g as (gc1);
@ -139,4 +141,11 @@ public class LogicalGenerate<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD
public int hashCode() {
return Objects.hash(generators, generatorOutput);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder();
builder.addUniformSlot(child(0).getLogicalProperties().getFunctionalDependencies());
return builder.build();
}
}

View File

@ -18,6 +18,8 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -25,6 +27,7 @@ import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Filter;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.ExpressionUtils;
import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
@ -35,6 +38,7 @@ import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
/**
* Logical Having plan
@ -114,6 +118,14 @@ public class LogicalHaving<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_T
return conjuncts.equals(other.conjuncts);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
Builder fdBuilder = new Builder(
child().getLogicalProperties().getFunctionalDependencies());
getConjuncts().forEach(e -> fdBuilder.addUniformSlot(ExpressionUtils.extractUniformSlot(e)));
return fdBuilder.build();
}
@Override
public String toString() {
return Utils.toSqlString("LogicalHaving", "predicates", getPredicate());

View File

@ -18,8 +18,10 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
@ -27,9 +29,13 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Supplier;
/**
* Logical Intersect.
@ -98,4 +104,27 @@ public class LogicalIntersect extends LogicalSetOperation {
return new LogicalIntersect(qualifier, newOutputs, regularChildrenOutputs,
Optional.empty(), Optional.empty(), children);
}
void replaceSlotInFuncDeps(FunctionalDependencies.Builder builder,
List<Slot> originalOutputs, List<Slot> newOutputs) {
Map<Slot, Slot> replaceMap = new HashMap<>();
for (int i = 0; i < newOutputs.size(); i++) {
replaceMap.put(originalOutputs.get(i), newOutputs.get(i));
}
builder.replace(replaceMap);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder();
for (Plan child : children) {
builder.addFunctionalDependencies(
child.getLogicalProperties().getFunctionalDependencies());
replaceSlotInFuncDeps(builder, child.getOutput(), outputSupplier.get());
}
if (qualifier == Qualifier.DISTINCT) {
builder.addUniqueSlot(ImmutableSet.copyOf(outputSupplier.get()));
}
return builder.build();
}
}

View File

@ -17,10 +17,14 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.common.Pair;
import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.rules.exploration.join.JoinReorderContext;
import org.apache.doris.nereids.trees.expressions.EqualTo;
import org.apache.doris.nereids.trees.expressions.ExprId;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.MarkJoinSlotReference;
@ -37,17 +41,19 @@ import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.json.JSONObject;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import javax.annotation.Nullable;
/**
* Logical join plan.
@ -269,7 +275,7 @@ public class LogicalJoin<LEFT_CHILD_TYPE extends Plan, RIGHT_CHILD_TYPE extends
@Override
public List<? extends Expression> getExpressions() {
return new Builder<Expression>()
return new ImmutableList.Builder<Expression>()
.addAll(hashJoinConjuncts)
.addAll(otherJoinConjuncts)
.build();
@ -357,6 +363,92 @@ public class LogicalJoin<LEFT_CHILD_TYPE extends Plan, RIGHT_CHILD_TYPE extends
markJoinSlotReference, children);
}
private @Nullable Pair<Set<Slot>, Set<Slot>> extractHashKeys() {
Set<Slot> leftKeys = new HashSet<>();
Set<Slot> rightKeys = new HashSet<>();
for (Expression expression : hashJoinConjuncts) {
// Note we don't support null-safe predicate right now, because we just check uniqueness for join keys
if (!(expression instanceof EqualTo
&& ((EqualTo) expression).left() instanceof Slot
&& ((EqualTo) expression).right() instanceof Slot)) {
return null;
}
Slot leftKey = (Slot) ((EqualTo) expression).left();
Slot rightKey = (Slot) ((EqualTo) expression).right();
if (left().getOutputSet().contains(leftKey)) {
leftKeys.add(leftKey);
rightKeys.add(rightKey);
} else {
leftKeys.add(rightKey);
rightKeys.add(leftKey);
}
}
return Pair.of(leftKeys, rightKeys);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
//1. NALAJ and FOJ block functional dependencies
if (joinType.isNullAwareLeftAntiJoin() || joinType.isFullOuterJoin()) {
return FunctionalDependencies.EMPTY_FUNC_DEPS;
}
// left/right semi/anti join propagate left/right functional dependencies
if (joinType.isLeftAntiJoin() || joinType.isLefSemiJoin()) {
return left().getLogicalProperties().getFunctionalDependencies();
}
if (joinType.isRightSemiJoin() || joinType.isRightAntiJoin()) {
return right().getLogicalProperties().getFunctionalDependencies();
}
// if there is non-equal join conditions, block functional dependencies
if (!otherJoinConjuncts.isEmpty()) {
return FunctionalDependencies.EMPTY_FUNC_DEPS;
}
Pair<Set<Slot>, Set<Slot>> keys = extractHashKeys();
if (keys == null) {
return FunctionalDependencies.EMPTY_FUNC_DEPS;
}
// Note here we only check whether the left is unique.
// So the hash condition can't be null-safe
// TODO: consider Null-safe hash condition when left and rigth is not nullable
boolean isLeftUnique = left().getLogicalProperties()
.getFunctionalDependencies().isUnique(keys.first);
boolean isRightUnique = left().getLogicalProperties()
.getFunctionalDependencies().isUnique(keys.first);
Builder fdBuilder = new Builder();
if (joinType.isInnerJoin()) {
// inner join propagate uniforms slots
// And if the hash keys is unique, inner join can propagate all functional dependencies
if (isLeftUnique && isRightUnique) {
fdBuilder.addFunctionalDependencies(left().getLogicalProperties().getFunctionalDependencies());
fdBuilder.addFunctionalDependencies(right().getLogicalProperties().getFunctionalDependencies());
} else {
fdBuilder.addUniformSlot(left().getLogicalProperties().getFunctionalDependencies());
fdBuilder.addUniformSlot(right().getLogicalProperties().getFunctionalDependencies());
}
}
// left/right outer join propagate left/right uniforms slots
// And if the right/left hash keys is unique,
// join can propagate left/right functional dependencies
if (joinType.isLeftOuterJoin()) {
if (isRightUnique) {
return left().getLogicalProperties().getFunctionalDependencies();
}
fdBuilder.addUniformSlot(left().getLogicalProperties().getFunctionalDependencies());
}
if (joinType.isRightOuterJoin()) {
if (isLeftUnique) {
return left().getLogicalProperties().getFunctionalDependencies();
}
fdBuilder.addUniformSlot(left().getLogicalProperties().getFunctionalDependencies());
}
return fdBuilder.build();
}
@Override
public JSONObject toJson() {
JSONObject logicalJoin = super.toJson();

View File

@ -18,6 +18,8 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -34,6 +36,7 @@ import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* Logical limit plan
@ -140,4 +143,16 @@ public class LogicalLimit<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TY
Preconditions.checkArgument(children.size() == 1);
return new LogicalLimit<>(limit, offset, phase, children.get(0));
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies fd = child(0).getLogicalProperties().getFunctionalDependencies();
if (getLimit() == 1 && !phase.isLocal()) {
Builder builder = new Builder();
outputSupplier.get().forEach(builder::addUniformSlot);
outputSupplier.get().forEach(builder::addUniqueSlot);
fd = builder.build();
}
return fd;
}
}

View File

@ -25,6 +25,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.algebra.Sink;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -39,7 +40,8 @@ import java.util.Optional;
/**
* logical olap table sink for insert command
*/
public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE> implements Sink {
public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE>
implements Sink, PropagateFuncDeps {
// bound data sink
private final Database database;
private final OlapTable targetTable;

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
@ -34,6 +35,7 @@ import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* A relation that contains only one row consist of some constant expressions.
@ -126,4 +128,14 @@ public class LogicalOneRowRelation extends LogicalRelation implements OneRowRela
public Plan pruneOutputs(List<NamedExpression> prunedOutputs) {
return withProjects(prunedOutputs);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder();
outputSupplier.get().forEach(s -> {
builder.addUniformSlot(s);
builder.addUniqueSlot(s);
});
return builder.build();
}
}

View File

@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.functions.window.Rank;
import org.apache.doris.nereids.trees.expressions.functions.window.RowNumber;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.WindowFuncType;
import org.apache.doris.nereids.trees.plans.algebra.PartitionTopN;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
@ -43,7 +44,8 @@ import java.util.Optional;
/**
* Logical partition-top-N plan.
*/
public class LogicalPartitionTopN<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements PartitionTopN {
public class LogicalPartitionTopN<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE>
implements PartitionTopN, PropagateFuncDeps {
private final WindowFuncType function;
private final List<Expression> partitionKeys;
private final List<OrderExpression> orderKeys;

View File

@ -17,10 +17,13 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Optional;
import java.util.function.BiFunction;
import java.util.function.Supplier;
@ -50,4 +53,11 @@ public interface LogicalPlan extends Plan {
return (LogicalPlan) withChildren(ImmutableList.copyOf(children()));
}
/**
* Compute FunctionalDependencies for different plan
* Note: Unless you really know what you're doing, please use the following interface.
* - BlockFDPropagation: clean the fd
* - PropagateFD: propagate the fd
*/
FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier);
}

View File

@ -20,11 +20,14 @@ package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.analyzer.Unbound;
import org.apache.doris.nereids.analyzer.UnboundStar;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.BoundStar;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Uuid;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.algebra.Project;
@ -34,11 +37,14 @@ import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.json.JSONObject;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* Logical project plan.
@ -200,4 +206,26 @@ public class LogicalProject<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_
logicalProject.put("Properties", properties);
return logicalProject;
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies childFuncDeps = child().getLogicalProperties().getFunctionalDependencies();
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder(childFuncDeps);
builder.pruneSlots(new HashSet<>(outputSupplier.get()));
projects.stream().filter(Alias.class::isInstance).forEach(proj -> {
if (proj.child(0).isConstant()) {
builder.addUniformSlot(proj.toSlot());
} else if (proj.child(0) instanceof Uuid) {
builder.addUniqueSlot(proj.toSlot());
} else if (ExpressionUtils.isInjective(proj.child(0))) {
ImmutableSet<Slot> inputs = ImmutableSet.copyOf(proj.getInputSlots());
if (childFuncDeps.isUnique(inputs)) {
builder.addUniqueSlot(proj.toSlot());
} else if (childFuncDeps.isUniform(inputs)) {
builder.addUniformSlot(proj.toSlot());
}
}
});
return builder.build();
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
@ -36,6 +37,7 @@ import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* LogicalRepeat.
@ -177,4 +179,14 @@ public class LogicalRepeat<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_T
return bound() && outputExpressions.stream()
.noneMatch(output -> output.containsType(VirtualSlotReference.class));
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder();
// Note uniform does not reject nullable slots
outputSupplier.get().stream()
.filter(child(0).getLogicalProperties().getFunctionalDependencies()::isUniform)
.forEach(builder::addUniformSlot);
return builder.build();
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.algebra.Sink;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -34,7 +35,8 @@ import java.util.Optional;
/**
* result sink
*/
public class LogicalResultSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE> implements Sink {
public class LogicalResultSink<CHILD_TYPE extends Plan> extends LogicalSink<CHILD_TYPE>
implements Sink, PropagateFuncDeps {
public LogicalResultSink(List<NamedExpression> outputExprs, CHILD_TYPE child) {
super(PlanType.LOGICAL_RESULT_SINK, outputExprs, child);

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.SelectHint;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
@ -40,7 +41,8 @@ import java.util.stream.Collectors;
* select hint plan.
* e.g. LogicalSelectHint (set_var(query_timeout='1800', exec_mem_limit='2147483648'))
*/
public class LogicalSelectHint<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> {
public class LogicalSelectHint<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE>
implements BlockFuncDepsPropagation {
private final Map<String, SelectHint> hints;

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.PropagateFuncDeps;
import org.apache.doris.nereids.trees.plans.algebra.Sort;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
@ -42,7 +43,8 @@ import java.util.Optional;
* orderKeys: list of column information after order by. eg:[a, asc],[b, desc].
* OrderKey: Contains order expression information and sorting method. Default is ascending.
*/
public class LogicalSort<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE> implements Sort {
public class LogicalSort<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYPE>
implements Sort, PropagateFuncDeps {
private final List<OrderKey> orderKeys;

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -30,9 +31,12 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.commons.lang3.StringUtils;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* The node of logical plan for sub query and alias
@ -153,4 +157,17 @@ public class LogicalSubQueryAlias<CHILD_TYPE extends Plan> extends LogicalUnary<
return new LogicalSubQueryAlias<>(qualifier, columnAliases, groupExpression, logicalProperties,
children.get(0));
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies
.Builder(child(0).getLogicalProperties().getFunctionalDependencies());
Map<Slot, Slot> replaceMap = new HashMap<>();
List<Slot> outputs = outputSupplier.get();
for (int i = 0; i < outputs.size(); i++) {
replaceMap.put(child(0).getOutput().get(i), outputs.get(i));
}
builder.replace(replaceMap);
return builder.build();
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
@ -37,7 +38,7 @@ import java.util.Objects;
import java.util.Optional;
/** LogicalTableValuedFunctionRelation */
public class LogicalTVFRelation extends LogicalRelation implements TVFRelation {
public class LogicalTVFRelation extends LogicalRelation implements TVFRelation, BlockFuncDepsPropagation {
private final TableValuedFunction function;
private final ImmutableList<String> qualifier;

View File

@ -18,6 +18,8 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.OrderKey;
import org.apache.doris.nereids.trees.expressions.Expression;
@ -34,6 +36,7 @@ import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* Logical top-N plan.
@ -148,4 +151,17 @@ public class LogicalTopN<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_TYP
Preconditions.checkArgument(children.size() == 1);
return new LogicalTopN<>(orderKeys, limit, offset, groupExpression, logicalProperties, children.get(0));
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies fd = child(0).getLogicalProperties().getFunctionalDependencies();
if (getLimit() == 1) {
Builder builder = new Builder();
List<Slot> output = outputSupplier.get();
output.forEach(builder::addUniformSlot);
output.forEach(builder::addUniqueSlot);
fd = builder.build();
}
return fd;
}
}

View File

@ -18,9 +18,11 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
@ -30,10 +32,12 @@ import org.apache.doris.nereids.util.Utils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* Logical Union.
@ -170,4 +174,14 @@ public class LogicalUnion extends LogicalSetOperation implements Union, OutputPr
public LogicalUnion pruneOutputs(List<NamedExpression> prunedOutputs) {
return withNewOutputs(prunedOutputs);
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
if (qualifier != Qualifier.DISTINCT) {
return FunctionalDependencies.EMPTY_FUNC_DEPS;
}
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder();
builder.addUniqueSlot(ImmutableSet.copyOf(outputSupplier.get()));
return builder.build();
}
}

View File

@ -18,6 +18,7 @@
package org.apache.doris.nereids.trees.plans.logical;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
@ -36,10 +37,12 @@ import org.apache.doris.qe.ConnectContext;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/**
* logical node to deal with window functions;
@ -224,4 +227,46 @@ public class LogicalWindow<CHILD_TYPE extends Plan> extends LogicalUnary<CHILD_T
return Optional.ofNullable(window);
}
private void updateFuncDepsByWindowExpr(NamedExpression namedExpression, FunctionalDependencies.Builder builder) {
if (namedExpression.children().size() != 1 || !(namedExpression.child(0) instanceof WindowExpression)) {
return;
}
WindowExpression windowExpr = (WindowExpression) namedExpression.child(0);
List<Expression> partitionKeys = windowExpr.getPartitionKeys();
// Now we only support slot type keys
if (!partitionKeys.stream().allMatch(Slot.class::isInstance)) {
return;
}
ImmutableSet<Slot> slotSet = partitionKeys.stream()
.map(s -> (Slot) s)
.collect(ImmutableSet.toImmutableSet());
// if partition by keys are unique, output is uniform
if (child(0).getLogicalProperties().getFunctionalDependencies().isUniqueAndNotNull(slotSet)) {
if (windowExpr.getFunction() instanceof RowNumber
|| windowExpr.getFunction() instanceof Rank
|| windowExpr.getFunction() instanceof DenseRank) {
builder.addUniformSlot(namedExpression.toSlot());
}
}
// if partition by keys are uniform, output is unique
if (child(0).getLogicalProperties().getFunctionalDependencies().isUniformAndNotNull(slotSet)) {
if (windowExpr.getFunction() instanceof RowNumber) {
builder.addUniqueSlot(namedExpression.toSlot());
}
}
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
FunctionalDependencies.Builder builder = new FunctionalDependencies.Builder(
child(0).getLogicalProperties().getFunctionalDependencies());
for (NamedExpression namedExpression : windowExpressions) {
updateFuncDepsByWindowExpr(namedExpression, builder);
}
return builder.build();
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.MarkJoinSlotReference;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation;
import org.apache.doris.nereids.trees.plans.JoinHint;
import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.Plan;
@ -41,7 +42,7 @@ import java.util.Optional;
* select col1 from t1 join t2 using(col1);
*/
public class UsingJoin<LEFT_CHILD_TYPE extends Plan, RIGHT_CHILD_TYPE extends Plan>
extends LogicalBinary<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> implements Join {
extends LogicalBinary<LEFT_CHILD_TYPE, RIGHT_CHILD_TYPE> implements Join, BlockFuncDepsPropagation {
private final JoinType joinType;
private final ImmutableList<Expression> otherJoinConjuncts;

View File

@ -37,6 +37,11 @@ import org.apache.doris.nereids.trees.expressions.Not;
import org.apache.doris.nereids.trees.expressions.Or;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
import org.apache.doris.nereids.trees.expressions.functions.agg.Avg;
import org.apache.doris.nereids.trees.expressions.functions.agg.Max;
import org.apache.doris.nereids.trees.expressions.functions.agg.Min;
import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.literal.NullLiteral;
@ -444,6 +449,32 @@ public class ExpressionUtils {
.collect(ImmutableSet.toImmutableSet());
}
/**
* extract uniform slot for the given predicate, such as a = 1 and b = 2
*/
public static ImmutableSet<Slot> extractUniformSlot(Expression expression) {
ImmutableSet.Builder<Slot> builder = new ImmutableSet.Builder<>();
if (expression instanceof And) {
builder.addAll(extractUniformSlot(expression.child(0)));
builder.addAll(extractUniformSlot(expression.child(1)));
}
if (expression instanceof EqualTo) {
if (isInjective(expression.child(0)) && expression.child(1).isConstant()) {
builder.add((Slot) expression.child(0));
}
}
return builder.build();
}
// TODO: Add more injective functions
public static boolean isInjective(Expression expression) {
return expression instanceof Slot;
}
public static boolean isInjectiveAgg(AggregateFunction agg) {
return agg instanceof Sum || agg instanceof Avg || agg instanceof Max || agg instanceof Min;
}
public static <E> Set<E> mutableCollect(List<? extends Expression> expressions,
Predicate<TreeNode<Expression>> predicate) {
return expressions.stream()

View File

@ -19,6 +19,7 @@ package org.apache.doris.nereids.glue.translator;
import org.apache.doris.catalog.KeysType;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.GreaterThan;
@ -63,7 +64,7 @@ public class PhysicalPlanTranslatorTest {
t1Output.add(col1);
t1Output.add(col2);
t1Output.add(col3);
LogicalProperties t1Properties = new LogicalProperties(() -> t1Output);
LogicalProperties t1Properties = new LogicalProperties(() -> t1Output, () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
PhysicalOlapScan scan = new PhysicalOlapScan(StatementScopeIdGenerator.newRelationId(), t1, qualifier, t1.getBaseIndexId(),
Collections.emptyList(), Collections.emptyList(), null, PreAggStatus.on(),
ImmutableList.of(), Optional.empty(), t1Properties, Optional.empty());

View File

@ -22,6 +22,7 @@ import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.PrimitiveType;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.jobs.JobContext;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.ExprId;
@ -87,7 +88,7 @@ public class DeriveStatsJobTest {
OlapTable table1 = PlanConstructor.newOlapTable(tableId1, "t1", 0);
return (LogicalOlapScan) new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), table1,
Collections.emptyList()).withGroupExprLogicalPropChildren(Optional.empty(),
Optional.of(new LogicalProperties(() -> ImmutableList.of(slot1))), ImmutableList.of());
Optional.of(new LogicalProperties(() -> ImmutableList.of(slot1), () -> FunctionalDependencies.EMPTY_FUNC_DEPS)), ImmutableList.of());
}
private LogicalAggregate constructAgg(Plan child) {

View File

@ -22,6 +22,7 @@ import org.apache.doris.common.jmockit.Deencapsulation;
import org.apache.doris.nereids.analyzer.UnboundRelation;
import org.apache.doris.nereids.analyzer.UnboundSlot;
import org.apache.doris.nereids.cost.Cost;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.properties.PhysicalProperties;
import org.apache.doris.nereids.properties.UnboundLogicalProperties;
@ -89,16 +90,18 @@ class MemoTest implements MemoPatternMatchSupported {
@Test
void testMergeGroup() {
Group srcGroup = new Group(new GroupId(2), new GroupExpression(new FakePlan()),
new LogicalProperties(ArrayList::new));
new LogicalProperties(ArrayList::new, () -> FunctionalDependencies.EMPTY_FUNC_DEPS));
Group dstGroup = new Group(new GroupId(3), new GroupExpression(new FakePlan()),
new LogicalProperties(ArrayList::new));
new LogicalProperties(ArrayList::new, () -> FunctionalDependencies.EMPTY_FUNC_DEPS));
FakePlan fakePlan = new FakePlan();
GroupExpression srcParentExpression = new GroupExpression(fakePlan, Lists.newArrayList(srcGroup));
Group srcParentGroup = new Group(new GroupId(0), srcParentExpression, new LogicalProperties(ArrayList::new));
Group srcParentGroup = new Group(new GroupId(0), srcParentExpression,
new LogicalProperties(ArrayList::new, () -> FunctionalDependencies.EMPTY_FUNC_DEPS));
srcParentGroup.setBestPlan(srcParentExpression, Cost.zeroV1(), PhysicalProperties.ANY);
GroupExpression dstParentExpression = new GroupExpression(fakePlan, Lists.newArrayList(dstGroup));
Group dstParentGroup = new Group(new GroupId(1), dstParentExpression, new LogicalProperties(ArrayList::new));
Group dstParentGroup = new Group(new GroupId(1), dstParentExpression,
new LogicalProperties(ArrayList::new, () -> FunctionalDependencies.EMPTY_FUNC_DEPS));
Memo memo = new Memo();
Map<GroupId, Group> groups = Deencapsulation.getField(memo, "groups");

View File

@ -21,6 +21,7 @@ import org.apache.doris.catalog.KeysType;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.processor.post.MergeProjectPostProcessor;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
@ -74,7 +75,7 @@ public class MergeProjectPostProcessTest {
t1Output.add(a);
t1Output.add(b);
t1Output.add(c);
LogicalProperties t1Properties = new LogicalProperties(() -> t1Output);
LogicalProperties t1Properties = new LogicalProperties(() -> t1Output, () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
PhysicalOlapScan scan = new PhysicalOlapScan(RelationId.createGenerator().getNextId(), t1, qualifier, 0L,
Collections.emptyList(), Collections.emptyList(), null, PreAggStatus.on(), ImmutableList.of(),
Optional.empty(), t1Properties, Optional.empty());

View File

@ -21,6 +21,7 @@ import org.apache.doris.catalog.KeysType;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.processor.post.PushDownFilterThroughProject;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.EqualTo;
@ -85,7 +86,7 @@ public class PushDownFilterThroughProjectTest {
t1Output.add(a);
t1Output.add(b);
t1Output.add(c);
LogicalProperties t1Properties = new LogicalProperties(() -> t1Output);
LogicalProperties t1Properties = new LogicalProperties(() -> t1Output, () -> FunctionalDependencies.EMPTY_FUNC_DEPS);
PhysicalOlapScan scan = new PhysicalOlapScan(RelationId.createGenerator().getNextId(), t1,
qualifier, 0L, Collections.emptyList(), Collections.emptyList(), null,
PreAggStatus.on(), ImmutableList.of(), Optional.empty(), t1Properties,

View File

@ -0,0 +1,387 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.properties;
import org.apache.doris.nereids.properties.FunctionalDependencies.Builder;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN;
import org.apache.doris.nereids.types.IntegerType;
import org.apache.doris.nereids.util.PlanChecker;
import org.apache.doris.utframe.TestWithFeService;
import com.google.common.collect.ImmutableSet;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
class FunctionalDependenciesTest extends TestWithFeService {
Slot slot1 = new SlotReference("1", IntegerType.INSTANCE, false);
Slot slot2 = new SlotReference("2", IntegerType.INSTANCE, false);
Slot slot3 = new SlotReference("1", IntegerType.INSTANCE, false);
Slot slot4 = new SlotReference("1", IntegerType.INSTANCE, false);
@Override
protected void runBeforeAll() throws Exception {
createDatabase("test");
createTable("create table test.agg (\n"
+ "id int not null,\n"
+ "name varchar(128) replace not null )\n"
+ "AGGREGATE KEY(id)\n"
+ "distributed by hash(id) buckets 10\n"
+ "properties('replication_num' = '1');");
createTable("create table test.uni (\n"
+ "id int not null,\n"
+ "name varchar(128) not null)\n"
+ "UNIQUE KEY(id)\n"
+ "distributed by hash(id) buckets 10\n"
+ "properties('replication_num' = '1');");
connectContext.setDatabase("default_cluster:test");
}
@Test
void testUniform() {
Builder fdBuilder = new Builder();
fdBuilder.addUniformSlot(slot1);
FunctionalDependencies fd = fdBuilder.build();
Assertions.assertTrue(fd.isUniformAndNotNull(slot1));
Assertions.assertFalse(fd.isUniformAndNotNull(slot2));
fdBuilder.addUniformSlot(ImmutableSet.of(slot2));
fd = fdBuilder.build();
Assertions.assertTrue(fd.isUniformAndNotNull(slot2));
ImmutableSet<Slot> slotSet = ImmutableSet.of(slot1, slot2, slot3);
fdBuilder.addUniformSlot(slotSet);
fd = fdBuilder.build();
Assertions.assertTrue(fd.isUniformAndNotNull(slotSet));
Assertions.assertFalse(fd.isUniformAndNotNull(ImmutableSet.of(slot1, slot2, slot3, slot4)));
Assertions.assertTrue(fd.isUniformAndNotNull(ImmutableSet.of(slot1, slot2)));
Assertions.assertFalse(fd.isUniformAndNotNull(ImmutableSet.of(slot3, slot2)));
}
@Test
void testUnique() {
Builder fdBuilder = new Builder();
fdBuilder.addUniqueSlot(slot1);
FunctionalDependencies fd = fdBuilder.build();
Assertions.assertTrue(fd.isUniqueAndNotNull(slot1));
Assertions.assertFalse(fd.isUniqueAndNotNull(slot2));
fdBuilder.addUniqueSlot(slot2);
fd = fdBuilder.build();
Assertions.assertTrue(fd.isUniqueAndNotNull(slot2));
ImmutableSet<Slot> slotSet = ImmutableSet.of(slot1, slot2, slot3);
fdBuilder.addUniqueSlot(slotSet);
fd = fdBuilder.build();
Assertions.assertTrue(fd.isUniqueAndNotNull(slotSet));
Assertions.assertTrue(fd.isUniqueAndNotNull(ImmutableSet.of(slot1, slot2, slot3, slot4)));
Assertions.assertFalse(fd.isUniqueAndNotNull(ImmutableSet.of(slot3, slot4)));
}
@Test
void testMergeFD() {
Builder fdBuilder1 = new Builder();
fdBuilder1.addUniformSlot(slot1);
Builder fdBuilder2 = new Builder();
fdBuilder2.addUniformSlot(slot2);
fdBuilder1.addFunctionalDependencies(fdBuilder2.build());
FunctionalDependencies fd = fdBuilder1.build();
Assertions.assertTrue(fd.isUniformAndNotNull(slot1));
Assertions.assertTrue(fd.isUniformAndNotNull(slot2));
}
@Test
void testScan() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select id from agg")
.rewrite()
.getPlan();
Assertions.assertTrue(plan.getLogicalProperties().getFunctionalDependencies()
.isUniqueAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select id from uni")
.rewrite()
.getPlan();
Assertions.assertTrue(plan.getLogicalProperties().getFunctionalDependencies()
.isUniqueAndNotNull(plan.getOutput().get(0)));
}
@Test
void testFilter() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select name from agg where name = \"1\"")
.rewrite()
.getPlan();
Assertions.assertTrue(plan.getLogicalProperties().getFunctionalDependencies()
.isUniformAndNotNull(plan.getOutput().get(0)));
}
@Test
void testJoin() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select * from agg full outer join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
Assertions.assertTrue(plan.getLogicalProperties().getFunctionalDependencies().isEmpty());
plan = PlanChecker.from(connectContext)
.analyze("select agg.id, uni.id from agg full outer join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
Assertions.assertTrue(plan.getLogicalProperties().getFunctionalDependencies().isEmpty());
plan = PlanChecker.from(connectContext)
.analyze("select agg.id from agg left outer join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select agg.id from agg left semi join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select agg.id from agg left anti join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select uni.id from agg right outer join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select uni.id, agg.id from agg inner join uni "
+ "on agg.id = uni.id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(1)));
}
@Test
void testAgg() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select count(1), name from agg group by name")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(1)));
plan = PlanChecker.from(connectContext)
.analyze("select count(1), max(id), id from agg group by id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(1)));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(2)));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select count(1), id from agg where id = 1 group by id")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(1)));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select count(name) from agg")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select id from agg where id = 1 group by cube(id, name)")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniform(plan.getOutput().get(0)));
}
@Test
void testGenerate() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select k1 from (select 1 k1) as t lateral view explode([1,2,3]) tmp1 as e1")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniform(plan.getOutput().get(0)));
}
@Test
void testWindow() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select row_number() over(partition by id) from agg")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select row_number() over(partition by name) from agg where name = '1'")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select row_number() over(partition by name) from agg where name = '1' limit 1")
.rewrite()
.getPlan();
LogicalPartitionTopN<?> ptopn = (LogicalPartitionTopN<?>) plan.child(0).child(0).child(0).child(0).child(0);
System.out.println(ptopn.getLogicalProperties().getFunctionalDependencies());
System.out.println(ptopn.getOutput());
Assertions.assertTrue(ptopn.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(ImmutableSet.copyOf(ptopn.getOutputSet())));
plan = PlanChecker.from(connectContext)
.analyze("select row_number() over(partition by name) from agg limit 1")
.rewrite()
.getPlan();
ptopn = (LogicalPartitionTopN<?>) plan.child(0).child(0).child(0).child(0).child(0);
Assertions.assertTrue(ptopn.getLogicalProperties()
.getFunctionalDependencies().isEmpty());
}
@Test
void testProject() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select name from agg where id = 1")
.rewrite()
.getPlan();
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isEmpty());
}
@Test
void testLimitAndTopN() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select name from agg limit 1")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(0)));
plan = PlanChecker.from(connectContext)
.analyze("select name from agg order by id limit 1")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(0)));
}
@Test
void testSubQuery() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select id from (select * from agg) t")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
}
@Disabled
@Test
void testCTE() {
Plan plan = PlanChecker.from(connectContext)
.analyze("with t as (select * from agg) select id from t where id = 1")
.getPlan();
System.out.println(plan.treeString());
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
}
@Test
void testSetOperation() {
Plan plan = PlanChecker.from(connectContext)
.analyze("select * from agg union select * from uni")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(ImmutableSet.copyOf(plan.getOutput())));
plan = PlanChecker.from(connectContext)
.analyze("select * from agg union all select * from uni")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertFalse(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(ImmutableSet.copyOf(plan.getOutput())));
plan = PlanChecker.from(connectContext)
.analyze("select * from agg intersect select * from uni where name = \"1\"")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(ImmutableSet.copyOf(plan.getOutput())));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniformAndNotNull(plan.getOutput().get(1)));
plan = PlanChecker.from(connectContext)
.analyze("select * from agg except select * from uni")
.rewrite()
.getPlan();
System.out.println(plan.getLogicalProperties().getFunctionalDependencies());
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(ImmutableSet.copyOf(plan.getOutput())));
Assertions.assertTrue(plan.getLogicalProperties()
.getFunctionalDependencies().isUniqueAndNotNull(plan.getOutput().get(0)));
}
}

View File

@ -20,6 +20,7 @@ package org.apache.doris.nereids.stats;
import org.apache.doris.common.IdGenerator;
import org.apache.doris.nereids.memo.Group;
import org.apache.doris.nereids.memo.GroupId;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.EqualTo;
import org.apache.doris.nereids.trees.expressions.Slot;
@ -73,14 +74,14 @@ public class JoinEstimateTest {
public List<Slot> get() {
return Lists.newArrayList(a);
}
})));
}, () -> FunctionalDependencies.EMPTY_FUNC_DEPS)));
GroupPlan right = new GroupPlan(new Group(idGenerator.getNextId(), new LogicalProperties(
new Supplier<List<Slot>>() {
@Override
public List<Slot> get() {
return Lists.newArrayList(b);
}
})));
}, () -> FunctionalDependencies.EMPTY_FUNC_DEPS)));
LogicalJoin join = new LogicalJoin(JoinType.INNER_JOIN, Lists.newArrayList(eq),
left, right);
Statistics outputStats = JoinEstimation.estimate(leftStats, rightStats, join);
@ -124,14 +125,14 @@ public class JoinEstimateTest {
public List<Slot> get() {
return Lists.newArrayList(a);
}
})));
}, () -> FunctionalDependencies.EMPTY_FUNC_DEPS)));
GroupPlan right = new GroupPlan(new Group(idGenerator.getNextId(), new LogicalProperties(
new Supplier<List<Slot>>() {
@Override
public List<Slot> get() {
return Lists.newArrayList(b, c);
}
})));
}, () -> FunctionalDependencies.EMPTY_FUNC_DEPS)));
LogicalJoin join = new LogicalJoin(JoinType.LEFT_OUTER_JOIN, Lists.newArrayList(eq),
left, right);
Statistics outputStats = JoinEstimation.estimate(leftStats, rightStats, join);

View File

@ -22,6 +22,7 @@ import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.PrimitiveType;
import org.apache.doris.nereids.memo.Group;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.EqualTo;
@ -61,7 +62,8 @@ public class StatsCalculatorTest {
private Group newFakeGroup() {
GroupExpression groupExpression = new GroupExpression(scan);
Group group = new Group(null, groupExpression, new LogicalProperties(Collections::emptyList));
Group group = new Group(null, groupExpression,
new LogicalProperties(Collections::emptyList, () -> FunctionalDependencies.EMPTY_FUNC_DEPS));
group.getLogicalExpressions().remove(0);
return group;
}
@ -250,7 +252,7 @@ public class StatsCalculatorTest {
LogicalOlapScan logicalOlapScan1 = (LogicalOlapScan) new LogicalOlapScan(
StatementScopeIdGenerator.newRelationId(), table1,
Collections.emptyList()).withGroupExprLogicalPropChildren(Optional.empty(),
Optional.of(new LogicalProperties(() -> ImmutableList.of(slot1))), ImmutableList.of());
Optional.of(new LogicalProperties(() -> ImmutableList.of(slot1), () -> FunctionalDependencies.EMPTY_FUNC_DEPS)), ImmutableList.of());
GroupExpression groupExpression = new GroupExpression(logicalOlapScan1, ImmutableList.of());
Group ownerGroup = new Group(null, groupExpression, null);