This is an automated email from the ASF dual-hosted git repository.
huajianlan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push:
new 7c7852994c (fix)(Nereids) fix ssb and add regression test case (#11095)
7c7852994c is described below
commit 7c7852994cba077e440c5c82e5bd6722018b709a
Author: 924060929 <[email protected]>
AuthorDate: Sat Jul 23 12:41:47 2022 +0800
(fix)(Nereids) fix ssb and add regression test case (#11095)
current nereids planner execute ssb will run into dead loop and crash be,
this pr fix this problem and add some regression test case prevent execute ssb
failed
---
.../glue/translator/ExpressionTranslator.java | 2 +-
.../glue/translator/PhysicalPlanTranslator.java | 11 ++++-
.../doris/nereids/jobs/batch/BatchRulesJob.java | 3 --
.../java/org/apache/doris/nereids/memo/Memo.java | 15 ++++---
.../java/org/apache/doris/nereids/rules/Rule.java | 5 +++
.../doris/nereids/trees/expressions/And.java | 7 ++++
.../apache/doris/nereids/trees/expressions/Or.java | 7 ++++
.../org/apache/doris/planner/HashJoinNode.java | 9 ++++-
.../org/apache/doris/regression/Config.groovy | 13 ++++--
.../doris/regression/action/TestAction.groovy | 47 +++++++++++++++++-----
.../doris/regression/suite/SuiteContext.groovy | 3 +-
.../doris/regression/util/OutputUtils.groovy | 6 ++-
regression-test/suites/ssb_sf1/nereids/q1.1.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q1.2.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q1.3.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q2.1.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q2.2.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q2.3.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q3.1.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q3.2.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q3.3.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q3.4.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q4.1.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q4.2.groovy | 36 +++++++++++++++++
regression-test/suites/ssb_sf1/nereids/q4.3.groovy | 36 +++++++++++++++++
25 files changed, 566 insertions(+), 30 deletions(-)
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
index 81934a9466..72cdb45f31 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
@@ -76,7 +76,7 @@ public class ExpressionTranslator extends
DefaultExpressionVisitor<Expr, PlanTra
* @return stale planner's expr
*/
public static Expr translate(Expression expression, PlanTranslatorContext
context) {
- Expr staleExpr = expression.accept(INSTANCE, context);
+ Expr staleExpr = expression.accept(INSTANCE, context);
try {
staleExpr.finalizeForNereids();
} catch (org.apache.doris.common.AnalysisException e) {
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
index aa23e63c1c..f6c8731526 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
@@ -354,9 +354,16 @@ public class PhysicalPlanTranslator extends
DefaultPlanVisitor<PlanFragment, Pla
.map(e -> ExpressionTranslator.translate(e, context))
.collect(Collectors.toList());
+ TupleDescriptor outputDescriptor = context.generateTupleDesc();
+ List<Expr> srcToOutput = hashJoin.getOutput().stream()
+ .map(SlotReference.class::cast)
+ .peek(s -> context.createSlotDesc(outputDescriptor, s))
+ .map(e -> ExpressionTranslator.translate(e, context))
+ .collect(Collectors.toList());
+
HashJoinNode hashJoinNode = new
HashJoinNode(context.nextPlanNodeId(), leftFragmentPlanRoot,
- rightFragmentPlanRoot,
- JoinType.toJoinOperator(joinType), execEqConjunctList,
Lists.newArrayList());
+ rightFragmentPlanRoot, JoinType.toJoinOperator(joinType),
execEqConjunctList, Lists.newArrayList(),
+ srcToOutput, outputDescriptor, outputDescriptor);
hashJoinNode.setDistributionMode(DistributionMode.BROADCAST);
hashJoinNode.setChild(0, leftFragmentPlanRoot);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/BatchRulesJob.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/BatchRulesJob.java
index 5510647431..beaa192dcc 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/BatchRulesJob.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/BatchRulesJob.java
@@ -26,7 +26,6 @@ import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleFactory;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import java.util.Objects;
@@ -48,7 +47,6 @@ public abstract class BatchRulesJob {
for (RuleFactory ruleFactory : ruleFactories) {
rules.addAll(ruleFactory.buildRules());
}
- Collections.reverse(rules);
return new RewriteBottomUpJob(
plannerContext.getMemo().getRoot(),
rules,
@@ -60,7 +58,6 @@ public abstract class BatchRulesJob {
for (RuleFactory ruleFactory : ruleFactories) {
rules.addAll(ruleFactory.buildRules());
}
- Collections.reverse(rules);
return new RewriteTopDownJob(
plannerContext.getMemo().getRoot(),
rules,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
index c6a42dee0d..e78737a618 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
@@ -132,10 +132,13 @@ public class Memo {
private GroupExpression insertOrRewriteGroupExpression(GroupExpression
groupExpression, Group target,
boolean rewrite, LogicalProperties logicalProperties) {
GroupExpression existedGroupExpression =
groupExpressions.get(groupExpression);
- if (existedGroupExpression != null
- &&
existedGroupExpression.getOwnerGroup().getLogicalProperties().equals(logicalProperties))
{
+ if (existedGroupExpression != null) {
+ Group mergedGroup = existedGroupExpression.getOwnerGroup();
if (target != null &&
!target.getGroupId().equals(existedGroupExpression.getOwnerGroup().getGroupId()))
{
- mergeGroup(target, existedGroupExpression.getOwnerGroup());
+ mergedGroup = mergeGroup(target,
existedGroupExpression.getOwnerGroup());
+ }
+ if (rewrite) {
+ mergedGroup.setLogicalProperties(logicalProperties);
}
return existedGroupExpression;
}
@@ -164,10 +167,11 @@ public class Memo {
*
* @param source source group
* @param destination destination group
+ * @return merged group
*/
- private void mergeGroup(Group source, Group destination) {
+ private Group mergeGroup(Group source, Group destination) {
if (source.equals(destination)) {
- return;
+ return source;
}
List<GroupExpression> needReplaceChild = Lists.newArrayList();
groupExpressions.values().forEach(groupExpression -> {
@@ -205,6 +209,7 @@ public class Memo {
source.movePhysicalExpressionOwnership(destination);
groups.remove(source);
}
+ return destination;
}
/**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
index 984414bdd8..05ddfbb134 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/Rule.java
@@ -62,5 +62,10 @@ public abstract class Rule {
return ruleType.getRuleTypeClass() == RuleTypeClass.REWRITE;
}
+ @Override
+ public String toString() {
+ return getRuleType().toString();
+ }
+
public abstract List<Plan> transform(Plan node, PlannerContext context)
throws TransformException;
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java
index 0a2bf62e58..0991c7f663 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/And.java
@@ -17,6 +17,8 @@
package org.apache.doris.nereids.trees.expressions;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+
import com.google.common.base.Preconditions;
import java.util.List;
@@ -41,6 +43,11 @@ public class And extends CompoundPredicate {
return new And(children.get(0), children.get(1));
}
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitAnd(this, context);
+ }
+
@Override
public CompoundPredicate flip() {
return new Or(left(), right());
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java
index 660a8bce8f..52ff529891 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Or.java
@@ -17,6 +17,8 @@
package org.apache.doris.nereids.trees.expressions;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+
import com.google.common.base.Preconditions;
import java.util.List;
@@ -41,6 +43,11 @@ public class Or extends CompoundPredicate {
return new Or(children.get(0), children.get(1));
}
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitOr(this, context);
+ }
+
@Override
public CompoundPredicate flip() {
return new And(left(), right());
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java
b/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java
index f835922cb8..bd7db83860 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HashJoinNode.java
@@ -60,6 +60,7 @@ import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
@@ -157,8 +158,9 @@ public class HashJoinNode extends PlanNode {
/**
* This constructor is used by new optimizer.
*/
- public HashJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner,
JoinOperator joinOp, List<Expr> eqJoinConjuncts,
- List<Expr> otherJoinConjuncts) {
+ public HashJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner,
JoinOperator joinOp,
+ List<Expr> eqJoinConjuncts, List<Expr> otherJoinConjuncts,
List<Expr> srcToOutputList,
+ TupleDescriptor intermediateTuple, TupleDescriptor outputTuple) {
super(id, "HASH JOIN", StatisticalType.HASH_JOIN_NODE);
Preconditions.checkArgument(eqJoinConjuncts != null &&
!eqJoinConjuncts.isEmpty());
Preconditions.checkArgument(otherJoinConjuncts != null);
@@ -209,6 +211,9 @@ public class HashJoinNode extends PlanNode {
} else if (joinOp.equals(JoinOperator.RIGHT_OUTER_JOIN)) {
nullableTupleIds.addAll(outer.getTupleIds());
}
+ vIntermediateTupleDescList = Lists.newArrayList(intermediateTuple);
+ vOutputTupleDesc = outputTuple;
+ vSrcToOutputSMap = new ExprSubstitutionMap(srcToOutputList,
Collections.emptyList());
}
public List<BinaryPredicate> getEqJoinConjuncts() {
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
index e368cf57b6..c8ddee08bd 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
@@ -371,13 +371,20 @@ class Config {
return DriverManager.getConnection(jdbcUrl, jdbcUser, jdbcPassword)
}
- Connection getConnection(String group) {
- String dbUrl = buildUrl(defaultDb + '_' + group)
- tryCreateDbIfNotExist(defaultDb + '_' + group)
+ Connection getConnectionByLastGroup(String group) {
+ String realDb = getDbByLastGroup(group)
+ String dbUrl = buildUrl(realDb)
+ tryCreateDbIfNotExist(realDb)
log.info("connect to ${dbUrl}".toString())
return DriverManager.getConnection(dbUrl, jdbcUser, jdbcPassword)
}
+ String getDbByLastGroup(String group) {
+ def groupList = group.split(',')
+ String lastGroup = groupList[groupList.length -
1].replace(File.separator, '_')
+ return defaultDb + '_' + lastGroup
+ }
+
Predicate<String> getDirectoryFilter() {
return (Predicate<String>) { String directoryName ->
if (directories.isEmpty() && excludeDirectorySet.isEmpty()) {
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/TestAction.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/TestAction.groovy
index bd03b76d9e..acf3034524 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/TestAction.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/TestAction.groovy
@@ -30,7 +30,6 @@ import org.apache.http.client.methods.RequestBuilder
import org.apache.http.impl.client.HttpClients
import org.apache.http.util.EntityUtils
-import javax.swing.text.html.parser.Entity
import java.nio.charset.StandardCharsets
import java.sql.Connection
import java.sql.ResultSetMetaData
@@ -46,6 +45,7 @@ class TestAction implements SuiteAction {
private String sql
private boolean isOrder
private String resultFileUri
+ private String resultTag
private Iterator<Object> resultIterator
private Object result
private long time
@@ -104,14 +104,34 @@ class TestAction implements SuiteAction {
}
if (this.resultFileUri != null) {
Consumer<InputStream> checkFunc = { InputStream
inputStream ->
+ String errorMsg = null
def lineIt = new LineIterator(new
InputStreamReader(inputStream, StandardCharsets.UTF_8))
- def csvIt = new OutputUtils.CsvParserIterator(lineIt)
- String errMsg = OutputUtils.checkOutput(csvIt,
result.result.iterator(),
- { List<String> row ->
OutputUtils.toCsvString(row as List<Object>) },
- { List<Object> row ->
OutputUtils.toCsvString(row) },
- "Check failed compare to", result.meta)
- if (errMsg != null) {
- throw new IllegalStateException(errMsg)
+ if (resultTag.is(null)) {
+ def csvIt = new
OutputUtils.CsvParserIterator(lineIt)
+ errorMsg = OutputUtils.checkOutput(csvIt,
result.result.iterator(),
+ { List<String> row ->
OutputUtils.toCsvString(row as List<Object>) },
+ { List<Object> row ->
OutputUtils.toCsvString(row) },
+ "Check failed compare to", result.meta)
+ if (errorMsg != null) {
+ throw new IllegalStateException(errorMsg)
+ }
+ } else {
+ def outputIt = OutputUtils.iterator(lineIt)
+ if (!outputIt.hasNextTagBlock(resultTag)) {
+ throw new IllegalStateException("Missing
output block for tag '${resultTag}': ${context.outputFile.getAbsolutePath()}")
+ }
+ def expectCsvResults = outputIt.next()
+ try {
+ errorMsg =
OutputUtils.checkOutput(expectCsvResults, result.result.iterator(),
+ { row -> OutputUtils.toCsvString(row
as List<Object>) },
+ { row -> OutputUtils.toCsvString(row)
},
+ "Check tag '${resultTag}' failed",
result.meta)
+ } catch (Throwable t) {
+ throw new IllegalStateException("Check tag
'${resultTag}' failed, sql:\n${sql}", t)
+ }
+ if (errorMsg != null) {
+ throw new IllegalStateException("Check tag
'${resultTag}' failed:\n${errorMsg}\n\nsql:\n${sql}")
+ }
}
}
@@ -132,7 +152,7 @@ class TestAction implements SuiteAction {
if (!new File(fileName).isAbsolute()) {
fileName = new File(context.dataPath,
fileName).getAbsolutePath()
}
- def file = new File(fileName)
+ def file = new File(fileName).getCanonicalFile()
if (!file.exists()) {
log.warn("Result file not exists:
${file}".toString())
}
@@ -213,14 +233,19 @@ class TestAction implements SuiteAction {
this.resultIterator = resultIteratorSupplier.call()
}
- void resultFile(String resultFile) {
- this.resultFileUri = resultFile
+ void resultFile(String file, String tag = null) {
+ this.resultFileUri = file
+ this.resultTag = tag
}
void resultFile(Closure<String> resultFileSupplier) {
this.resultFileUri = resultFileSupplier.call()
}
+ void resultTag(String tag) {
+ this.resultTag = tag
+ }
+
void exception(String exceptionMsg) {
this.exception = exceptionMsg
}
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
index 7608c21b44..f463f72e4c 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
@@ -108,8 +108,7 @@ class SuiteContext implements Closeable {
Connection getConnection() {
def threadConn = threadLocalConn.get()
if (threadConn == null) {
- def groupList = group.split(',')
- threadConn = config.getConnection(groupList[groupList.length -
1].replace(File.separator, '_'))
+ threadConn = config.getConnectionByLastGroup(group)
threadLocalConn.set(threadConn)
}
return threadConn
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
index b220f4deeb..957f3fb570 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
@@ -148,7 +148,11 @@ class OutputUtils {
}
static OutputBlocksIterator iterator(File file) {
- def it = new ReusableIterator<String>(new LineIteratorAdaptor(new
LineIterator(new FileReader(file))))
+ return iterator(new LineIterator(new FileReader(file)));
+ }
+
+ static OutputBlocksIterator iterator(LineIterator closeableIterator) {
+ def it = new ReusableIterator<String>(new
LineIteratorAdaptor(closeableIterator))
return new OutputBlocksIterator(it)
}
diff --git a/regression-test/suites/ssb_sf1/nereids/q1.1.groovy
b/regression-test/suites/ssb_sf1/nereids/q1.1.groovy
new file mode 100644
index 0000000000..2383510c06
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q1.1.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q1_1_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q1.1.sql").text)
+
+ resultFile(file = "../sql/q1.1.out", tag = "q1.1")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q1.2.groovy
b/regression-test/suites/ssb_sf1/nereids/q1.2.groovy
new file mode 100644
index 0000000000..cb6a7f742f
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q1.2.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q1_2_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q1.2.sql").text)
+
+ resultFile(file = "../sql/q1.2.out", tag = "q1.2")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q1.3.groovy
b/regression-test/suites/ssb_sf1/nereids/q1.3.groovy
new file mode 100644
index 0000000000..068897e1bd
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q1.3.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q1_3_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q1.3.sql").text)
+
+ resultFile(file = "../sql/q1.3.out", tag = "q1.3")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q2.1.groovy
b/regression-test/suites/ssb_sf1/nereids/q2.1.groovy
new file mode 100644
index 0000000000..1a8103559c
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q2.1.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q2_1_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q2.1.sql").text)
+
+ resultFile(file = "../sql/q2.1.out", tag = "q2.1")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q2.2.groovy
b/regression-test/suites/ssb_sf1/nereids/q2.2.groovy
new file mode 100644
index 0000000000..290f9de260
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q2.2.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q2_2_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q2.2.sql").text)
+
+ resultFile(file = "../sql/q2.2.out", tag = "q2.2")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q2.3.groovy
b/regression-test/suites/ssb_sf1/nereids/q2.3.groovy
new file mode 100644
index 0000000000..bb71a1d214
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q2.3.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q2_3_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q2.3.sql").text)
+
+ resultFile(file = "../sql/q2.3.out", tag = "q2.3")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q3.1.groovy
b/regression-test/suites/ssb_sf1/nereids/q3.1.groovy
new file mode 100644
index 0000000000..cc0486bc77
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q3.1.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q3_1_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q3.1.sql").text)
+
+ resultFile(file = "../sql/q3.1.out", tag = "q3.1")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q3.2.groovy
b/regression-test/suites/ssb_sf1/nereids/q3.2.groovy
new file mode 100644
index 0000000000..c25454d604
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q3.2.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q3_2_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q3.2.sql").text)
+
+ resultFile(file = "../sql/q3.2.out", tag = "q3.2")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q3.3.groovy
b/regression-test/suites/ssb_sf1/nereids/q3.3.groovy
new file mode 100644
index 0000000000..c4e4dbd5e6
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q3.3.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q3_3_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q3.3.sql").text)
+
+ resultFile(file = "../sql/q3.3.out", tag = "q3.3")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q3.4.groovy
b/regression-test/suites/ssb_sf1/nereids/q3.4.groovy
new file mode 100644
index 0000000000..47b3faa314
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q3.4.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q3_4_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q3.4.sql").text)
+
+ resultFile(file = "../sql/q3.4.out", tag = "q3.4")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q4.1.groovy
b/regression-test/suites/ssb_sf1/nereids/q4.1.groovy
new file mode 100644
index 0000000000..2d93598782
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q4.1.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q4_1_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q4.1.sql").text)
+
+ resultFile(file = "../sql/q4.1.out", tag = "q4.1")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q4.2.groovy
b/regression-test/suites/ssb_sf1/nereids/q4.2.groovy
new file mode 100644
index 0000000000..34e485c53c
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q4.2.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q4_2_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q4.2.sql").text)
+
+ resultFile(file = "../sql/q4.2.out", tag = "q4.2")
+ }
+}
diff --git a/regression-test/suites/ssb_sf1/nereids/q4.3.groovy
b/regression-test/suites/ssb_sf1/nereids/q4.3.groovy
new file mode 100644
index 0000000000..90c844cb53
--- /dev/null
+++ b/regression-test/suites/ssb_sf1/nereids/q4.3.groovy
@@ -0,0 +1,36 @@
+// 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.
+
+suite("ssb_sf1_q4_3_nereids") {
+ String realDb = context.config.getDbByLastGroup(context.group)
+ // get parent directory's group
+ realDb = realDb.substring(0, realDb.lastIndexOf("_"))
+
+ sql "use ${realDb}"
+
+ sql 'set enable_nereids_planner=true'
+ // nereids need vectorized
+ sql 'set enable_vectorized_engine=true'
+
+ sql 'set exec_mem_limit=2147483648*2'
+
+ test {
+ sql(new File(context.file.parentFile, "../sql/q4.3.sql").text)
+
+ resultFile(file = "../sql/q4.3.out", tag = "q4.3")
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]