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]

Reply via email to