[enhancement](Nereids) compare LogicalProperties with output set instead of output list (#12743)
We used output list to compare two LogicalProperties before. Since join reorder will change the children order of a join plan and caused output list changed. the two join plan will not equals anymore in memo although they should be. So we must add a project on the new join to keep the LogicalProperties the same. This PR changes the equals and hashCode funtions of LogicalProperties. use a set of output to compare two LogicalProperties. Then we do not need add the top peoject anymore. This help us keep memo simple and efficient.
This commit is contained in:
@ -549,7 +549,6 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
|
||||
.map(TupleDescriptor::getSlots)
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
TupleDescriptor outputDescriptor = context.generateTupleDesc();
|
||||
Map<ExprId, SlotReference> outputSlotReferenceMap = Maps.newHashMap();
|
||||
|
||||
hashJoin.getOutput().stream()
|
||||
@ -579,6 +578,15 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
|
||||
.map(SlotReference.class::cast)
|
||||
.forEach(s -> hashOutputSlotReferenceMap.put(s.getExprId(), s));
|
||||
|
||||
Map<ExprId, SlotReference> leftChildOutputMap = Maps.newHashMap();
|
||||
hashJoin.child(0).getOutput().stream()
|
||||
.map(SlotReference.class::cast)
|
||||
.forEach(s -> leftChildOutputMap.put(s.getExprId(), s));
|
||||
Map<ExprId, SlotReference> rightChildOutputMap = Maps.newHashMap();
|
||||
hashJoin.child(1).getOutput().stream()
|
||||
.map(SlotReference.class::cast)
|
||||
.forEach(s -> rightChildOutputMap.put(s.getExprId(), s));
|
||||
|
||||
//make intermediate tuple
|
||||
List<SlotDescriptor> leftIntermediateSlotDescriptor = Lists.newArrayList();
|
||||
List<SlotDescriptor> rightIntermediateSlotDescriptor = Lists.newArrayList();
|
||||
@ -586,47 +594,43 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
|
||||
|
||||
if (!hashJoin.getOtherJoinCondition().isPresent()
|
||||
&& (joinType == JoinType.LEFT_ANTI_JOIN || joinType == JoinType.LEFT_SEMI_JOIN)) {
|
||||
leftIntermediateSlotDescriptor = hashJoin.child(0).getOutput().stream()
|
||||
.map(SlotReference.class::cast)
|
||||
.map(s -> context.createSlotDesc(intermediateDescriptor, s))
|
||||
.collect(Collectors.toList());
|
||||
for (SlotDescriptor leftSlotDescriptor : leftSlotDescriptors) {
|
||||
SlotReference sf = leftChildOutputMap.get(context.findExprId(leftSlotDescriptor.getId()));
|
||||
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
|
||||
leftIntermediateSlotDescriptor.add(sd);
|
||||
}
|
||||
} else if (!hashJoin.getOtherJoinCondition().isPresent()
|
||||
&& (joinType == JoinType.RIGHT_ANTI_JOIN || joinType == JoinType.RIGHT_SEMI_JOIN)) {
|
||||
rightIntermediateSlotDescriptor = hashJoin.child(1).getOutput().stream()
|
||||
.map(SlotReference.class::cast)
|
||||
.map(s -> context.createSlotDesc(intermediateDescriptor, s))
|
||||
.collect(Collectors.toList());
|
||||
for (SlotDescriptor rightSlotDescriptor : rightSlotDescriptors) {
|
||||
SlotReference sf = rightChildOutputMap.get(context.findExprId(rightSlotDescriptor.getId()));
|
||||
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
|
||||
rightIntermediateSlotDescriptor.add(sd);
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < hashJoin.child(0).getOutput().size(); i++) {
|
||||
SlotReference sf = (SlotReference) hashJoin.child(0).getOutput().get(i);
|
||||
for (SlotDescriptor leftSlotDescriptor : leftSlotDescriptors) {
|
||||
SlotReference sf = leftChildOutputMap.get(context.findExprId(leftSlotDescriptor.getId()));
|
||||
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
|
||||
if (hashOutputSlotReferenceMap.get(sf.getExprId()) != null) {
|
||||
hashJoinNode.addSlotIdToHashOutputSlotIds(leftSlotDescriptors.get(i).getId());
|
||||
hashJoinNode.addSlotIdToHashOutputSlotIds(leftSlotDescriptor.getId());
|
||||
}
|
||||
leftIntermediateSlotDescriptor.add(sd);
|
||||
}
|
||||
for (int i = 0; i < hashJoin.child(1).getOutput().size(); i++) {
|
||||
SlotReference sf = (SlotReference) hashJoin.child(1).getOutput().get(i);
|
||||
for (SlotDescriptor rightSlotDescriptor : rightSlotDescriptors) {
|
||||
SlotReference sf = rightChildOutputMap.get(context.findExprId(rightSlotDescriptor.getId()));
|
||||
SlotDescriptor sd = context.createSlotDesc(intermediateDescriptor, sf);
|
||||
if (hashOutputSlotReferenceMap.get(sf.getExprId()) != null) {
|
||||
hashJoinNode.addSlotIdToHashOutputSlotIds(rightSlotDescriptors.get(i).getId());
|
||||
hashJoinNode.addSlotIdToHashOutputSlotIds(rightSlotDescriptor.getId());
|
||||
}
|
||||
rightIntermediateSlotDescriptor.add(sd);
|
||||
}
|
||||
}
|
||||
|
||||
//set slots as nullable for outer join
|
||||
if (joinType == JoinType.FULL_OUTER_JOIN) {
|
||||
rightIntermediateSlotDescriptor.stream()
|
||||
.forEach(sd -> sd.setIsNullable(true));
|
||||
leftIntermediateSlotDescriptor.stream()
|
||||
.forEach(sd -> sd.setIsNullable(true));
|
||||
} else if (joinType == JoinType.LEFT_OUTER_JOIN) {
|
||||
rightIntermediateSlotDescriptor.stream()
|
||||
.forEach(sd -> sd.setIsNullable(true));
|
||||
} else if (joinType == JoinType.RIGHT_OUTER_JOIN) {
|
||||
leftIntermediateSlotDescriptor.stream()
|
||||
.forEach(sd -> sd.setIsNullable(true));
|
||||
if (joinType == JoinType.LEFT_OUTER_JOIN || joinType == JoinType.FULL_OUTER_JOIN) {
|
||||
rightIntermediateSlotDescriptor.forEach(sd -> sd.setIsNullable(true));
|
||||
}
|
||||
if (joinType == JoinType.RIGHT_OUTER_JOIN || joinType == JoinType.FULL_OUTER_JOIN) {
|
||||
leftIntermediateSlotDescriptor.forEach(sd -> sd.setIsNullable(true));
|
||||
}
|
||||
|
||||
List<Expr> otherJoinConjuncts = hashJoin.getOtherJoinCondition()
|
||||
@ -658,7 +662,8 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
|
||||
.map(e -> ExpressionTranslator.translate(e, context))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
outputSlotReferences.stream().forEach(s -> context.createSlotDesc(outputDescriptor, s));
|
||||
TupleDescriptor outputDescriptor = context.generateTupleDesc();
|
||||
outputSlotReferences.forEach(s -> context.createSlotDesc(outputDescriptor, s));
|
||||
|
||||
hashJoinNode.setvOutputTupleDesc(outputDescriptor);
|
||||
hashJoinNode.setvSrcToOutputSMap(srcToOutput);
|
||||
|
||||
@ -489,9 +489,7 @@ public class Memo {
|
||||
List<Plan> groupPlanChildren = childrenGroups.stream()
|
||||
.map(GroupPlan::new)
|
||||
.collect(ImmutableList.toImmutableList());
|
||||
LogicalProperties logicalProperties = plan.getLogicalProperties();
|
||||
return plan.withChildren(groupPlanChildren)
|
||||
.withLogicalProperties(Optional.of(logicalProperties));
|
||||
return plan.withChildren(groupPlanChildren);
|
||||
}
|
||||
|
||||
/*
|
||||
|
||||
@ -17,19 +17,24 @@
|
||||
|
||||
package org.apache.doris.nereids.properties;
|
||||
|
||||
import org.apache.doris.nereids.trees.expressions.ExprId;
|
||||
import org.apache.doris.nereids.trees.expressions.NamedExpression;
|
||||
import org.apache.doris.nereids.trees.expressions.Slot;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Logical properties used for analysis and optimize in Nereids.
|
||||
*/
|
||||
public class LogicalProperties {
|
||||
protected Supplier<List<Slot>> outputSupplier;
|
||||
protected final Supplier<List<Slot>> outputSupplier;
|
||||
protected final Supplier<HashSet<ExprId>> outputSetSupplier;
|
||||
|
||||
/**
|
||||
* constructor of LogicalProperties.
|
||||
@ -41,6 +46,10 @@ public class LogicalProperties {
|
||||
this.outputSupplier = Suppliers.memoize(
|
||||
Objects.requireNonNull(outputSupplier, "outputSupplier can not be null")
|
||||
);
|
||||
this.outputSetSupplier = Suppliers.memoize(
|
||||
() -> outputSupplier.get().stream().map(NamedExpression::getExprId)
|
||||
.collect(Collectors.toCollection(HashSet::new))
|
||||
);
|
||||
}
|
||||
|
||||
public List<Slot> getOutput() {
|
||||
@ -60,11 +69,11 @@ public class LogicalProperties {
|
||||
return false;
|
||||
}
|
||||
LogicalProperties that = (LogicalProperties) o;
|
||||
return Objects.equals(outputSupplier.get(), that.outputSupplier.get());
|
||||
return Objects.equals(outputSetSupplier.get(), that.outputSetSupplier.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(outputSupplier.get());
|
||||
return Objects.hash(outputSetSupplier.get());
|
||||
}
|
||||
}
|
||||
|
||||
@ -20,7 +20,6 @@ package org.apache.doris.nereids.rules;
|
||||
import org.apache.doris.nereids.rules.exploration.join.InnerJoinLAsscom;
|
||||
import org.apache.doris.nereids.rules.exploration.join.InnerJoinLAsscomProject;
|
||||
import org.apache.doris.nereids.rules.exploration.join.JoinCommute;
|
||||
import org.apache.doris.nereids.rules.exploration.join.JoinCommuteProject;
|
||||
import org.apache.doris.nereids.rules.exploration.join.OuterJoinLAsscom;
|
||||
import org.apache.doris.nereids.rules.exploration.join.OuterJoinLAsscomProject;
|
||||
import org.apache.doris.nereids.rules.exploration.join.SemiJoinLogicalJoinTranspose;
|
||||
@ -57,7 +56,6 @@ import java.util.List;
|
||||
public class RuleSet {
|
||||
public static final List<Rule> EXPLORATION_RULES = planRuleFactories()
|
||||
.add(JoinCommute.LEFT_DEEP)
|
||||
.add(JoinCommuteProject.LEFT_DEEP)
|
||||
.add(InnerJoinLAsscom.INSTANCE)
|
||||
.add(InnerJoinLAsscomProject.INSTANCE)
|
||||
.add(OuterJoinLAsscom.INSTANCE)
|
||||
|
||||
@ -23,9 +23,6 @@ import org.apache.doris.nereids.rules.exploration.OneExplorationRuleFactory;
|
||||
import org.apache.doris.nereids.rules.exploration.join.JoinCommuteHelper.SwapType;
|
||||
import org.apache.doris.nereids.trees.plans.GroupPlan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
|
||||
import org.apache.doris.nereids.util.PlanUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* Join Commute
|
||||
@ -58,7 +55,7 @@ public class JoinCommute extends OneExplorationRuleFactory {
|
||||
newJoin.getJoinReorderContext().setHasCommuteZigZag(true);
|
||||
}
|
||||
|
||||
return PlanUtils.project(new ArrayList<>(join.getOutput()), newJoin).get();
|
||||
return newJoin;
|
||||
}).toRule(RuleType.LOGICAL_JOIN_COMMUTATE);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1,68 +0,0 @@
|
||||
// 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.rules.exploration.join;
|
||||
|
||||
import org.apache.doris.nereids.rules.Rule;
|
||||
import org.apache.doris.nereids.rules.RuleType;
|
||||
import org.apache.doris.nereids.rules.exploration.OneExplorationRuleFactory;
|
||||
import org.apache.doris.nereids.rules.exploration.join.JoinCommuteHelper.SwapType;
|
||||
import org.apache.doris.nereids.trees.plans.GroupPlan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
|
||||
import org.apache.doris.nereids.util.PlanUtils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* Project-Join Commute.
|
||||
* This rule can prevent double JoinCommute cause dead-loop in Memo.
|
||||
*/
|
||||
public class JoinCommuteProject extends OneExplorationRuleFactory {
|
||||
|
||||
public static final JoinCommuteProject LEFT_DEEP = new JoinCommuteProject(SwapType.LEFT_DEEP);
|
||||
public static final JoinCommuteProject ZIG_ZAG = new JoinCommuteProject(SwapType.ZIG_ZAG);
|
||||
public static final JoinCommuteProject BUSHY = new JoinCommuteProject(SwapType.BUSHY);
|
||||
|
||||
private final SwapType swapType;
|
||||
|
||||
public JoinCommuteProject(SwapType swapType) {
|
||||
this.swapType = swapType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Rule build() {
|
||||
return logicalProject(logicalJoin())
|
||||
.when(project -> JoinCommuteHelper.check(swapType, project.child()))
|
||||
.then(project -> {
|
||||
LogicalJoin<GroupPlan, GroupPlan> join = project.child();
|
||||
// prevent this join match by JoinCommute.
|
||||
join.getGroupExpression().get().setApplied(RuleType.LOGICAL_JOIN_COMMUTATE);
|
||||
LogicalJoin<GroupPlan, GroupPlan> newJoin = new LogicalJoin<>(
|
||||
join.getJoinType().swap(),
|
||||
join.getHashJoinConjuncts(),
|
||||
join.getOtherJoinCondition(),
|
||||
join.right(), join.left(),
|
||||
join.getJoinReorderContext());
|
||||
newJoin.getJoinReorderContext().setHasCommute(true);
|
||||
if (swapType == SwapType.ZIG_ZAG && JoinCommuteHelper.isNotBottomJoin(join)) {
|
||||
newJoin.getJoinReorderContext().setHasCommuteZigZag(true);
|
||||
}
|
||||
|
||||
return PlanUtils.project(new ArrayList<>(project.getProjects()), newJoin).get();
|
||||
}).toRule(RuleType.LOGICAL_JOIN_COMMUTATE);
|
||||
}
|
||||
}
|
||||
@ -96,6 +96,10 @@ class JoinLAsscomHelper extends ThreeJoinHelper {
|
||||
topJoin.getJoinReorderContext());
|
||||
newTopJoin.getJoinReorderContext().setHasLAsscom(true);
|
||||
|
||||
if (topJoin.getLogicalProperties().equals(newTopJoin.getLogicalProperties())) {
|
||||
return newTopJoin;
|
||||
}
|
||||
|
||||
return PlanUtils.projectOrSelf(new ArrayList<>(topJoin.getOutput()), newTopJoin);
|
||||
}
|
||||
|
||||
|
||||
@ -87,9 +87,9 @@ public class InnerJoinLAsscomProjectTest {
|
||||
Assertions.assertEquals(2, root.getLogicalExpressions().size());
|
||||
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(0).getPlan() instanceof LogicalJoin);
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalProject);
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalJoin);
|
||||
|
||||
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1).child(0).getLogicalExpression();
|
||||
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1);
|
||||
GroupExpression leftProjectGroupExpr = newTopJoinGroupExpr.child(0).getLogicalExpression();
|
||||
GroupExpression rightProjectGroupExpr = newTopJoinGroupExpr.child(1).getLogicalExpression();
|
||||
Plan leftProject = newTopJoinGroupExpr.child(0).getLogicalExpression().getPlan();
|
||||
|
||||
@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.plans.JoinType;
|
||||
import org.apache.doris.nereids.trees.plans.Plan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
|
||||
import org.apache.doris.nereids.util.MemoTestUtils;
|
||||
import org.apache.doris.nereids.util.PlanChecker;
|
||||
import org.apache.doris.nereids.util.PlanConstructor;
|
||||
@ -76,9 +75,9 @@ public class InnerJoinLAsscomTest {
|
||||
Assertions.assertEquals(2, root.getLogicalExpressions().size());
|
||||
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(0).getPlan() instanceof LogicalJoin);
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalProject);
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalJoin);
|
||||
|
||||
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1).child(0).getLogicalExpression();
|
||||
GroupExpression newTopJoinGroupExpr = root.logicalExpressionsAt(1);
|
||||
GroupExpression newBottomJoinGroupExpr = newTopJoinGroupExpr.child(0).getLogicalExpression();
|
||||
Plan bottomLeft = newBottomJoinGroupExpr.child(0).getLogicalExpression().getPlan();
|
||||
Plan bottomRight = newBottomJoinGroupExpr.child(1).getLogicalExpression().getPlan();
|
||||
|
||||
@ -25,7 +25,6 @@ import org.apache.doris.nereids.trees.plans.Plan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
|
||||
import org.apache.doris.nereids.util.LogicalPlanBuilder;
|
||||
import org.apache.doris.nereids.util.MemoTestUtils;
|
||||
import org.apache.doris.nereids.util.PlanChecker;
|
||||
@ -51,9 +50,9 @@ public class JoinCommuteTest {
|
||||
Assertions.assertEquals(2, root.getLogicalExpressions().size());
|
||||
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(0).getPlan() instanceof LogicalJoin);
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalProject);
|
||||
Assertions.assertTrue(root.logicalExpressionsAt(1).getPlan() instanceof LogicalJoin);
|
||||
|
||||
GroupExpression newJoinGroupExpr = root.logicalExpressionsAt(1).child(0).getLogicalExpression();
|
||||
GroupExpression newJoinGroupExpr = root.logicalExpressionsAt(1);
|
||||
Plan left = newJoinGroupExpr.child(0).getLogicalExpression().getPlan();
|
||||
Plan right = newJoinGroupExpr.child(1).getLogicalExpression().getPlan();
|
||||
Assertions.assertTrue(left instanceof LogicalOlapScan);
|
||||
|
||||
@ -1,17 +1,17 @@
|
||||
-- This file is automatically generated. You should know what you did if you want to edit this
|
||||
-- !scalar_less_than_corr --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
3 3
|
||||
3 4
|
||||
|
||||
-- !scalar_not_equal_corr --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
@ -19,10 +19,10 @@
|
||||
-- !scalar_equal_to_corr --
|
||||
|
||||
-- !not_in_corr --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
2 5
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
@ -35,11 +35,11 @@
|
||||
2 5
|
||||
|
||||
-- !exist_corr --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
@ -49,108 +49,108 @@
|
||||
-- !exist_and_not_exist --
|
||||
|
||||
-- !scalar_unCorrelated --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
|
||||
-- !scalar_equal_to_uncorr --
|
||||
|
||||
-- !not_scalar_unCorrelated --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !scalar_not_equal_uncorr --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !in_unCorrelated --
|
||||
2 5
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
|
||||
-- !in_subquery_uncorr --
|
||||
2 5
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
|
||||
-- !not_in_unCorrelated --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !not_in_uncorr --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !exist_unCorrelated --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !exist_uncorr --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
2 5
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !alias_scalar --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
|
||||
-- !alias_in --
|
||||
3 3
|
||||
|
||||
-- !alias_not_in --
|
||||
1 3
|
||||
1 2
|
||||
2 5
|
||||
1 3
|
||||
2 4
|
||||
2 5
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
-- !alias_exist --
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
2 4
|
||||
3 4
|
||||
3 3
|
||||
3 4
|
||||
20 2
|
||||
22 3
|
||||
24 4
|
||||
|
||||
@ -21,8 +21,8 @@
|
||||
|
||||
-- !select_5 --
|
||||
1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK 1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK
|
||||
1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK 1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL
|
||||
1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL 1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK
|
||||
1309892 1 1303 1432 15 19920517 3-MEDIUM 0 24 2959704 5119906 7 2752524 73992 0 19920619 TRUCK 1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL
|
||||
1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL 1309892 2 1303 1165 9 19920517 3-MEDIUM 0 21 2404899 5119906 8 2212507 68711 7 19920616 RAIL
|
||||
1310179 6 1312 1455 29 19921110 3-MEDIUM 0 15 1705830 20506457 10 1535247 68233 8 19930114 FOB 1310179 6 1312 1455 29 19921110 3-MEDIUM 0 15 1705830 20506457 10 1535247 68233 8 19930114 FOB
|
||||
|
||||
|
||||
@ -20,10 +20,19 @@ suite ("sub_query_correlated") {
|
||||
sql """
|
||||
SET enable_vectorized_engine=true
|
||||
"""
|
||||
|
||||
sql """
|
||||
SET enable_nereids_planner=true
|
||||
"""
|
||||
|
||||
sql """
|
||||
SET enable_bucket_shuffle_join=false
|
||||
"""
|
||||
|
||||
sql """
|
||||
SET disable_colocate_plan=true
|
||||
"""
|
||||
|
||||
sql """
|
||||
DROP TABLE IF EXISTS `subquery1`
|
||||
"""
|
||||
@ -93,133 +102,133 @@ suite ("sub_query_correlated") {
|
||||
|
||||
//------------------Correlated-----------------
|
||||
qt_scalar_less_than_corr """
|
||||
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1
|
||||
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_scalar_not_equal_corr """
|
||||
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1
|
||||
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_scalar_equal_to_corr """
|
||||
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1
|
||||
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_not_in_corr """
|
||||
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1
|
||||
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_in_subquery_corr """
|
||||
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1
|
||||
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = subquery1.k2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_not_exist_corr """
|
||||
select * from subquery1 where not exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1
|
||||
select * from subquery1 where not exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_exist_corr """
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_in_with_in_and_scalar """
|
||||
select * from subquery1 where subquery1.k1 in (
|
||||
select subquery3.k3 from subquery3 where
|
||||
subquery3.k3 in (select subquery4.k1 from subquery4 where subquery4.k1 = 3)
|
||||
and subquery3.v2 > (select sum(subquery2.k2) from subquery2 where subquery2.k2 = subquery3.v1)) order by k1
|
||||
and subquery3.v2 > (select sum(subquery2.k2) from subquery2 where subquery2.k2 = subquery3.v1)) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_exist_and_not_exist """
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery1.k2 = subquery3.v2)
|
||||
and not exists (select subquery4.k2 from subquery4 where subquery1.k2 = subquery4.k2) order by k1
|
||||
and not exists (select subquery4.k2 from subquery4 where subquery1.k2 = subquery4.k2) order by k1, k2
|
||||
"""
|
||||
|
||||
//------------------unCorrelated-----------------
|
||||
qt_scalar_unCorrelated """
|
||||
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1
|
||||
select * from subquery1 where subquery1.k1 < (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_scalar_equal_to_uncorr """
|
||||
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3) order by k1
|
||||
select * from subquery1 where subquery1.k1 = (select sum(subquery3.k3) from subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_not_scalar_unCorrelated """
|
||||
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1
|
||||
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3 where subquery3.v2 = 2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_scalar_not_equal_uncorr """
|
||||
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3) order by k1
|
||||
select * from subquery1 where subquery1.k1 != (select sum(subquery3.k3) from subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_in_unCorrelated """
|
||||
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1
|
||||
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_in_subquery_uncorr """
|
||||
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3) order by k1
|
||||
select * from subquery1 where subquery1.k1 in (select subquery3.k3 from subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_not_in_unCorrelated """
|
||||
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1
|
||||
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_not_in_uncorr """
|
||||
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3) order by k1
|
||||
select * from subquery1 where subquery1.k1 not in (select subquery3.k3 from subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_exist_unCorrelated """
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3 where subquery3.v2 = 2) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_exist_uncorr """
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3) order by k1
|
||||
select * from subquery1 where exists (select subquery3.k3 from subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
//----------with subquery alias----------
|
||||
qt_alias_scalar """
|
||||
select * from subquery1
|
||||
where subquery1.k1 < (select max(aa) from
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_alias_in """
|
||||
select * from subquery1
|
||||
where subquery1.k1 in (select aa from
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_alias_not_in """
|
||||
select * from subquery1
|
||||
where subquery1.k1 not in (select aa from
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_alias_exist """
|
||||
select * from subquery1
|
||||
where exists (select aa from
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
qt_alias_not_exist """
|
||||
select * from subquery1
|
||||
where not exists (select aa from
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1
|
||||
(select k1 as aa from subquery3 where subquery1.k2 = subquery3.v2) subquery3) order by k1, k2
|
||||
"""
|
||||
|
||||
//----------complex subqueries----------
|
||||
qt_scalar_subquery """
|
||||
select * from subquery1
|
||||
where k1 = (select sum(k1) from subquery3 where subquery1.k1 = subquery3.v1 and subquery3.v2 = 2)
|
||||
order by k1
|
||||
order by k1, k2
|
||||
"""
|
||||
|
||||
qt_in_subquery """
|
||||
select * from subquery3
|
||||
where (k1 = 1 or k1 = 2 or k1 = 3) and v1 in (select k1 from subquery1 where subquery1.k2 = subquery3.v2 and subquery1.k1 = 3)
|
||||
order by k1
|
||||
order by k1, k2
|
||||
"""
|
||||
|
||||
qt_exist_subquery """
|
||||
select * from subquery3
|
||||
where k1 = 2 and exists (select * from subquery1 where subquery1.k1 = subquery3.v2 and subquery1.k2 = 4)
|
||||
order by k1;
|
||||
order by k1, k2
|
||||
"""
|
||||
}
|
||||
|
||||
@ -24,6 +24,14 @@ suite("view") {
|
||||
SET enable_nereids_planner=true
|
||||
"""
|
||||
|
||||
sql """
|
||||
SET enable_nereids_planner=true
|
||||
"""
|
||||
|
||||
sql """
|
||||
SET enable_bucket_shuffle_join=false
|
||||
"""
|
||||
|
||||
sql """
|
||||
create view if not exists v1 as
|
||||
select *
|
||||
|
||||
Reference in New Issue
Block a user