[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:
@ -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;
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
@ -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());
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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");
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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)));
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
Reference in New Issue
Block a user