This is an automated email from the ASF dual-hosted git repository.

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-2.0 by this push:
     new f0e3e207dc Revert "[Bug](agg-state) fix core dump on not nullable 
argument for aggstate's nested argument (#21331)"
f0e3e207dc is described below

commit f0e3e207dc60cf27b67200c5fe4554d729bb0838
Author: Kang <[email protected]>
AuthorDate: Tue Jul 4 00:27:47 2023 +0800

    Revert "[Bug](agg-state) fix core dump on not nullable argument for 
aggstate's nested argument (#21331)"
    
    This reverts commit dd2df6c81fcecc99c06e0a4f52b5dfebf318a923.
---
 be/src/vec/exprs/vectorized_agg_fn.cpp             |  21 +-
 .../org/apache/doris/analysis/ArithmeticExpr.java  |   2 +-
 .../apache/doris/analysis/NativeInsertStmt.java    |  22 +-
 .../doris/catalog/MaterializedIndexMeta.java       |   3 +
 .../functions/AggStateFunctionBuilder.java         |  13 +-
 .../functions/combinator/MergeCombinator.java      |   6 +-
 .../functions/combinator/StateCombinator.java      |   2 +-
 .../functions/combinator/UnionCombinator.java      |   6 +-
 .../apache/doris/nereids/types/AggStateType.java   |  11 +-
 .../java/org/apache/doris/analysis/ExprTest.java   |   2 +-
 .../org/apache/doris/analysis/LoadStmtTest.java    |   4 +-
 .../apache/doris/analysis/S3TvfLoadStmtTest.java   | 245 ---------------------
 .../org/apache/doris/planner/RepeatNodeTest.java   |   4 +-
 .../agg_state/max/test_agg_state_max.out           |  33 ---
 regression-test/data/mv_p0/k1ap2spa/k1ap2spa.out   |   2 -
 .../agg_state/max/test_agg_state_max.groovy        |  27 +--
 .../suites/mv_p0/k1ap2spa/k1ap2spa.groovy          |   1 -
 17 files changed, 45 insertions(+), 359 deletions(-)

diff --git a/be/src/vec/exprs/vectorized_agg_fn.cpp 
b/be/src/vec/exprs/vectorized_agg_fn.cpp
index 2dcf12207a..a6895ff728 100644
--- a/be/src/vec/exprs/vectorized_agg_fn.cpp
+++ b/be/src/vec/exprs/vectorized_agg_fn.cpp
@@ -168,12 +168,7 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const 
RowDescriptor& desc,
                     "Agg state function input type must be agg_state but get 
{}",
                     argument_types[0]->get_family_name());
         }
-
-        std::string type_function_name =
-                assert_cast<const DataTypeAggState*>(argument_types[0].get())
-                        ->get_nested_function()
-                        ->get_name();
-        if (type_function_name + AGG_UNION_SUFFIX == _fn.name.function_name) {
+        if (match_suffix(_fn.name.function_name, AGG_UNION_SUFFIX)) {
             if (_data_type->is_nullable()) {
                 return Status::InternalError(
                         "Union function return type must be not nullable, 
real={}",
@@ -185,19 +180,11 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const 
RowDescriptor& desc,
                         _data_type->get_name());
             }
             _function = 
get_agg_state_function<AggregateStateUnion>(argument_types, _data_type);
-        } else if (type_function_name + AGG_MERGE_SUFFIX == 
_fn.name.function_name) {
-            auto type = assert_cast<const 
DataTypeAggState*>(argument_types[0].get())
-                                ->get_nested_function()
-                                ->get_return_type();
-            if (!type->equals(*_data_type)) {
-                return Status::InternalError("{}'s expect return type is {}, 
but input {}",
-                                             argument_types[0]->get_name(), 
type->get_name(),
-                                             _data_type->get_name());
-            }
+        } else if (match_suffix(_fn.name.function_name, AGG_MERGE_SUFFIX)) {
             _function = 
get_agg_state_function<AggregateStateMerge>(argument_types, _data_type);
         } else {
-            return Status::InternalError("{} not match function {}", 
argument_types[0]->get_name(),
-                                         _fn.name.function_name);
+            return Status::InternalError(
+                    "Aggregate Function {} is not endwith '_merge' or 
'_union'", _fn.signature);
         }
     } else {
         _function = AggregateFunctionSimpleFactory::instance().get(
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
index 176a55fb02..de437e154c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ArithmeticExpr.java
@@ -260,7 +260,7 @@ public class ArithmeticExpr extends Expr {
         if (children.size() == 1) {
             return op.toString() + " " + getChild(0).toSql();
         } else {
-            return "(" + getChild(0).toSql() + " " + op.toString() + " " + 
getChild(1).toSql() + ")";
+            return getChild(0).toSql() + " " + op.toString() + " " + 
getChild(1).toSql();
         }
     }
 
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java
index f041ac81f1..d3956de245 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/NativeInsertStmt.java
@@ -420,7 +420,9 @@ public class NativeInsertStmt extends InsertStmt {
                 mentionedColumns.add(col.getName());
                 targetColumns.add(col);
             }
+            realTargetColumnNames = targetColumns.stream().map(column -> 
column.getName()).collect(Collectors.toList());
         } else {
+            realTargetColumnNames = targetColumnNames;
             for (String colName : targetColumnNames) {
                 Column col = targetTable.getColumn(colName);
                 if (col == null) {
@@ -434,8 +436,8 @@ public class NativeInsertStmt extends InsertStmt {
             // hll column mush in mentionedColumns
             for (Column col : targetTable.getBaseSchema()) {
                 if (col.getType().isObjectStored() && 
!mentionedColumns.contains(col.getName())) {
-                    throw new AnalysisException(
-                            " object-stored column " + col.getName() + " mush 
in insert into columns");
+                    throw new AnalysisException(" object-stored column " + 
col.getName()
+                            + " mush in insert into columns");
                 }
             }
         }
@@ -516,30 +518,20 @@ public class NativeInsertStmt extends InsertStmt {
         }
 
         // check if size of select item equal with columns mentioned in 
statement
-        if (mentionedColumns.size() != queryStmt.getResultExprs().size()) {
+        if (mentionedColumns.size() != queryStmt.getResultExprs().size()
+                || realTargetColumnNames.size() != 
queryStmt.getResultExprs().size()) {
             
ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_VALUE_COUNT);
         }
 
         // Check if all columns mentioned is enough
         checkColumnCoverage(mentionedColumns, targetTable.getBaseSchema());
 
-        realTargetColumnNames = targetColumns.stream().map(column -> 
column.getName()).collect(Collectors.toList());
         Map<String, Expr> slotToIndex = 
Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
-        for (int i = 0; i < queryStmt.getResultExprs().size(); i++) {
+        for (int i = 0; i < realTargetColumnNames.size(); i++) {
             slotToIndex.put(realTargetColumnNames.get(i), 
queryStmt.getResultExprs().get(i)
                     
.checkTypeCompatibility(targetTable.getColumn(realTargetColumnNames.get(i)).getType()));
         }
 
-        for (Column column : targetTable.getBaseSchema()) {
-            if (!slotToIndex.containsKey(column.getName())) {
-                if (column.getDefaultValue() == null) {
-                    slotToIndex.put(column.getName(), new NullLiteral());
-                } else {
-                    slotToIndex.put(column.getName(), new 
StringLiteral(column.getDefaultValue()));
-                }
-            }
-        }
-
         // handle VALUES() or SELECT constant list
         if (isValuesOrConstantSelect) {
             SelectStmt selectStmt = (SelectStmt) queryStmt;
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
index 12cab9f7ec..07c33b85bc 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
@@ -193,6 +193,9 @@ public class MaterializedIndexMeta implements Writable, 
GsonPostProcessable {
                 // mv_count_sale_amt -> mva_SUM__CASE WHEN `sale_amt` IS NULL 
THEN 0 ELSE 1 END
                 List<SlotRef> slots = new ArrayList<>();
                 entry.getValue().collect(SlotRef.class, slots);
+                if (slots.size() > 1) {
+                    throw new IOException("DefineExpr have multiple slot in 
MaterializedIndex, Expr=" + entry.getKey());
+                }
 
                 String name = 
MaterializedIndexMeta.normalizeName(slots.get(0).toSqlWithoutTbl());
                 Column matchedColumn = null;
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AggStateFunctionBuilder.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AggStateFunctionBuilder.java
index 054aa4767b..e07f263040 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AggStateFunctionBuilder.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/AggStateFunctionBuilder.java
@@ -18,12 +18,15 @@
 package org.apache.doris.nereids.trees.expressions.functions;
 
 import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.SlotReference;
 import 
org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
 import 
org.apache.doris.nereids.trees.expressions.functions.combinator.MergeCombinator;
 import 
org.apache.doris.nereids.trees.expressions.functions.combinator.StateCombinator;
 import 
org.apache.doris.nereids.trees.expressions.functions.combinator.UnionCombinator;
 import org.apache.doris.nereids.types.AggStateType;
 
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 import java.util.Objects;
 import java.util.stream.Collectors;
@@ -63,7 +66,9 @@ public class AggStateFunctionBuilder extends FunctionBuilder {
                 return false;
             }
 
-            return nestedBuilder.canApply(((AggStateType) 
argument.getDataType()).getMockedExpressions());
+            return nestedBuilder.canApply(((AggStateType) 
argument.getDataType()).getSubTypes().stream().map(t -> {
+                return new SlotReference("mocked", t);
+            }).collect(ImmutableList.toImmutableList()));
         }
     }
 
@@ -90,7 +95,11 @@ public class AggStateFunctionBuilder extends FunctionBuilder 
{
         Expression arg = (Expression) arguments.get(0);
         AggStateType type = (AggStateType) arg.getDataType();
 
-        return (AggregateFunction) nestedBuilder.build(nestedName, 
type.getMockedExpressions());
+        List<Expression> nestedArgumens = type.getSubTypes().stream().map(t -> 
{
+            return new SlotReference("mocked", t);
+        }).collect(Collectors.toList());
+
+        return (AggregateFunction) nestedBuilder.build(nestedName, 
nestedArgumens);
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/MergeCombinator.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/MergeCombinator.java
index b529ae2de3..69b9514f13 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/MergeCombinator.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/MergeCombinator.java
@@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableList;
 
 import java.util.List;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 /**
  * AggState combinator merge
@@ -46,12 +47,13 @@ public class MergeCombinator extends AggregateFunction
         super(nested.getName() + AggStateFunctionBuilder.MERGE_SUFFIX, 
arguments);
 
         this.nested = Objects.requireNonNull(nested, "nested can not be null");
-        inputType = (AggStateType) arguments.get(0).getDataType();
+        inputType = new AggStateType(nested.getName(), 
nested.getArgumentsTypes(),
+                
nested.getArguments().stream().map(Expression::nullable).collect(Collectors.toList()));
     }
 
     @Override
     public MergeCombinator withChildren(List<Expression> children) {
-        return new MergeCombinator(children, nested);
+        return new MergeCombinator(children, nested.withChildren(children));
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/StateCombinator.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/StateCombinator.java
index 9b97a7afd4..d7da86c525 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/StateCombinator.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/StateCombinator.java
@@ -59,7 +59,7 @@ public class StateCombinator extends ScalarFunction
 
     @Override
     public StateCombinator withChildren(List<Expression> children) {
-        return new StateCombinator(children, nested);
+        return new StateCombinator(children, nested.withChildren(children));
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/UnionCombinator.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/UnionCombinator.java
index 67f09a50eb..21261998ec 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/UnionCombinator.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/combinator/UnionCombinator.java
@@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableList;
 
 import java.util.List;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 /**
  * AggState combinator union
@@ -46,12 +47,13 @@ public class UnionCombinator extends AggregateFunction
         super(nested.getName() + AggStateFunctionBuilder.UNION_SUFFIX, 
arguments);
 
         this.nested = Objects.requireNonNull(nested, "nested can not be null");
-        inputType = (AggStateType) arguments.get(0).getDataType();
+        inputType = new AggStateType(nested.getName(), 
nested.getArgumentsTypes(),
+                
nested.getArguments().stream().map(Expression::nullable).collect(Collectors.toList()));
     }
 
     @Override
     public UnionCombinator withChildren(List<Expression> children) {
-        return new UnionCombinator(children, nested);
+        return new UnionCombinator(children, nested.withChildren(children));
     }
 
     @Override
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java
index a999512d1b..cc467b0eb6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java
@@ -19,14 +19,11 @@ package org.apache.doris.nereids.types;
 
 import org.apache.doris.analysis.Expr;
 import org.apache.doris.catalog.Type;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.types.coercion.AbstractDataType;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 import java.util.stream.Collectors;
@@ -59,12 +56,8 @@ public class AggStateType extends DataType {
         this.functionName = functionName;
     }
 
-    public List<Expression> getMockedExpressions() {
-        List<Expression> result = new ArrayList<Expression>();
-        for (int i = 0; i < subTypes.size(); i++) {
-            result.add(new SlotReference("mocked", subTypes.get(i), 
subTypeNullables.get(i)));
-        }
-        return result;
+    public List<DataType> getSubTypes() {
+        return subTypes;
     }
 
     @Override
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java
index a91199189a..5262a29034 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ExprTest.java
@@ -264,7 +264,7 @@ public class ExprTest {
         DataInputStream dis = new DataInputStream(new FileInputStream(file));
         Expr readExpr = Expr.readIn(dis);
         Assert.assertTrue(readExpr instanceof ArithmeticExpr);
-        Assert.assertEquals("(cos(1) + (100 / 200))", readExpr.toSql());
+        Assert.assertEquals("cos(1) + 100 / 200", readExpr.toSql());
 
         // 3. delete files
         dis.close();
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/analysis/LoadStmtTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/analysis/LoadStmtTest.java
index e8f8309867..59070dc80c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/LoadStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/LoadStmtTest.java
@@ -150,13 +150,13 @@ public class LoadStmtTest {
         columnDescs.descs = columns1;
         columnDescs.isColumnDescsRewrited = false;
         Load.rewriteColumns(columnDescs);
-        String orig = "((`c1` + 1) + 1)";
+        String orig = "`c1` + 1 + 1";
         Assert.assertEquals(orig, columns1.get(4).getExpr().toString());
 
         List<ImportColumnDesc> columns2 = getColumns("c1,c2,c3,tmp_c5 = 
tmp_c4+1, tmp_c4=c1 + 1");
         columnDescs.descs = columns2;
         columnDescs.isColumnDescsRewrited = false;
-        String orig2 = "(`tmp_c4` + 1)";
+        String orig2 = "`tmp_c4` + 1";
         Load.rewriteColumns(columnDescs);
         Assert.assertEquals(orig2, columns2.get(3).getExpr().toString());
 
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java
deleted file mode 100644
index cc41e31e3c..0000000000
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/S3TvfLoadStmtTest.java
+++ /dev/null
@@ -1,245 +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.analysis;
-
-import org.apache.doris.analysis.BinaryPredicate.Operator;
-import org.apache.doris.analysis.StorageBackend.StorageType;
-import org.apache.doris.catalog.Column;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
-import org.apache.doris.catalog.Table;
-import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.FeConstants;
-import org.apache.doris.common.UserException;
-import org.apache.doris.common.jmockit.Deencapsulation;
-import org.apache.doris.common.util.SqlParserUtils;
-import org.apache.doris.datasource.property.constants.S3Properties;
-import org.apache.doris.datasource.property.constants.S3Properties.Env;
-import org.apache.doris.load.loadv2.LoadTask.MergeType;
-import org.apache.doris.tablefunction.S3TableValuedFunction;
-
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import mockit.Expectations;
-import mockit.Injectable;
-import org.apache.hadoop.util.Lists;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.StringReader;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class S3TvfLoadStmtTest {
-
-    private static final String ACCESS_KEY_VALUE = "ak";
-
-    private static final String SECRET_KEY_VALUE = "sk";
-
-    private static final String ENDPOINT_VALUE = "cos.ap-beijing.myqcloud.com";
-
-    private static final String REGION_VALUE = "ap-beijing";
-
-    private static final String DATA_URI = 
"s3://doris-build-1308700295/regression/load/data/part*";
-
-    private static final String FORMAT = "parquet";
-
-    private static final String TARGET_TABLE_NAME = "target";
-
-    private LabelName labelName;
-
-    private BrokerDesc brokerDesc;
-
-    private Set<String> colNames;
-
-    @Before
-    public void setUp() throws AnalysisException {
-        FeConstants.runningUnitTest = true;
-
-        labelName = new LabelName("testDb", "testTbl");
-
-        final Map<String, String> brokerProperties = Maps.newHashMap();
-        brokerProperties.put(Env.ACCESS_KEY, ACCESS_KEY_VALUE);
-        brokerProperties.put(Env.SECRET_KEY, SECRET_KEY_VALUE);
-        brokerProperties.put(Env.ENDPOINT, ENDPOINT_VALUE);
-        brokerProperties.put(Env.REGION, REGION_VALUE);
-        brokerDesc = new BrokerDesc("s3", StorageType.S3, brokerProperties);
-
-        colNames = Sets.newHashSet("k1", "k2", "k3", "k4");
-    }
-
-    @Test
-    public void testClauses() throws UserException {
-        final BinaryPredicate greater = new BinaryPredicate(Operator.GT, new 
IntLiteral(1), new IntLiteral(0));
-        final BinaryPredicate less = new BinaryPredicate(Operator.LT, new 
IntLiteral(1), new IntLiteral(0));
-        DataDescription dataDescription = buildDataDesc(
-                Lists.newArrayList(colNames),
-                greater,
-                less,
-                null
-        );
-        final S3TvfLoadStmt s3TvfLoadStmt = new S3TvfLoadStmt(labelName, 
Collections.singletonList(dataDescription),
-                brokerDesc,
-                Maps.newHashMap(), "comment");
-        final SelectStmt selectStmt = (SelectStmt) 
s3TvfLoadStmt.getQueryStmt();
-        final Expr whereClause = Deencapsulation.getField(selectStmt, 
"whereClause");
-        Assert.assertEquals(whereClause, new 
CompoundPredicate(CompoundPredicate.Operator.AND, greater, less));
-    }
-
-    @Test
-    public void testTvfGeneration() {
-        DataDescription dataDescription = buildDataDesc(
-                Lists.newArrayList(colNames),
-                null,
-                null,
-                null
-        );
-        final TableRef tvfRef = Deencapsulation.invoke(S3TvfLoadStmt.class,
-                "buildTvfRef",
-                dataDescription, brokerDesc);
-        Assert.assertTrue(tvfRef instanceof TableValuedFunctionRef);
-        final S3TableValuedFunction tableFunction
-                = (S3TableValuedFunction) ((TableValuedFunctionRef) 
tvfRef).getTableFunction();
-        final Map<String, String> locationProperties = 
tableFunction.getLocationProperties();
-        Assert.assertEquals(locationProperties.get(S3Properties.ENDPOINT), 
ENDPOINT_VALUE);
-        Assert.assertEquals(locationProperties.get(S3Properties.ACCESS_KEY), 
ACCESS_KEY_VALUE);
-        Assert.assertEquals(locationProperties.get(S3Properties.SECRET_KEY), 
SECRET_KEY_VALUE);
-        Assert.assertEquals(locationProperties.get(S3Properties.REGION), 
REGION_VALUE);
-        Assert.assertEquals(tableFunction.getFilePath(), DATA_URI);
-    }
-
-    @Injectable
-    Table targetTable;
-
-    @Injectable
-    DataDescription dataDescription;
-
-    @Test
-    public void testColumnMappings() throws Exception {
-        // c1/c2/c3 in both file and table, and c5 is only in table
-        final List<ImportColumnDesc> columnsDescList = getColumnsDescList(
-                "c1,c2,c3,c1=upper(c1), tmp_c4=c1 + 1, c5 = tmp_c4+1");
-        //        DataDescription dataDescription = buildDataDesc(colNames, 
null, null, null);
-        new Expectations() {
-            {
-                dataDescription.getParsedColumnExprList();
-                minTimes = 0;
-                result = columnsDescList;
-
-                dataDescription.getFilePaths();
-                minTimes = 0;
-                result = Collections.singletonList(DATA_URI);
-
-                targetTable.getBaseSchema();
-                minTimes = 0;
-                result = getBaseSchema();
-
-                targetTable.getColumn("c1");
-                minTimes = 0;
-                result = new Column();
-
-                targetTable.getColumn("c2");
-                minTimes = 0;
-                result = new Column();
-
-                targetTable.getColumn("c3");
-                minTimes = 0;
-                result = new Column();
-
-                targetTable.getColumn("c5");
-                minTimes = 0;
-                result = new Column();
-
-                targetTable.getColumn("tmp_c4");
-                minTimes = 0;
-                result = null;
-            }
-        };
-        final S3TvfLoadStmt s3TvfLoadStmt = new S3TvfLoadStmt(labelName, 
Collections.singletonList(dataDescription),
-                brokerDesc, null, "comment");
-        s3TvfLoadStmt.setTargetTable(targetTable);
-        Deencapsulation.setField(s3TvfLoadStmt, "functionGenTableColNames", 
Sets.newHashSet("c1", "c2", "c3"));
-
-        Deencapsulation.invoke(s3TvfLoadStmt, "rewriteExpr", columnsDescList);
-        Assert.assertEquals(columnsDescList.size(), 5);
-        final String orig4 = "((upper(`c1`) + 1) + 1)";
-        Assert.assertEquals(orig4, 
columnsDescList.get(4).getExpr().toString());
-
-        final List<ImportColumnDesc> filterColumns = 
Deencapsulation.invoke(s3TvfLoadStmt,
-                "filterColumns", columnsDescList);
-        Assert.assertEquals(filterColumns.size(), 4);
-    }
-
-    private static DataDescription buildDataDesc(Iterable<String> columns, 
Expr fileFilter, Expr wherePredicate,
-            List<Expr> mappingList) {
-
-        return new DataDescription(
-                TARGET_TABLE_NAME,
-                null,
-                Collections.singletonList(DATA_URI),
-                Lists.newArrayList(columns),
-                null,
-                FORMAT,
-                null,
-                false,
-                mappingList,
-                fileFilter,
-                wherePredicate,
-                MergeType.APPEND,
-                null,
-                null,
-                null
-        );
-    }
-
-    private static List<ImportColumnDesc> getColumnsDescList(String columns) 
throws Exception {
-        String columnsSQL = "COLUMNS (" + columns + ")";
-        return ((ImportColumnsStmt) SqlParserUtils.getFirstStmt(
-                new SqlParser(new SqlScanner(new 
StringReader(columnsSQL))))).getColumns();
-    }
-
-    private static List<Column> getBaseSchema() {
-        List<Column> columns = com.google.common.collect.Lists.newArrayList();
-
-        Column c1 = new Column("c1", PrimitiveType.BIGINT);
-        c1.setIsKey(true);
-        c1.setIsAllowNull(false);
-        columns.add(c1);
-
-        Column c2 = new Column("c2", ScalarType.createVarchar(25));
-        c2.setIsKey(true);
-        c2.setIsAllowNull(true);
-        columns.add(c2);
-
-        Column c3 = new Column("c3", PrimitiveType.BIGINT);
-        c3.setIsKey(true);
-        c3.setIsAllowNull(false);
-        columns.add(c3);
-
-        Column c5 = new Column("c5", PrimitiveType.BIGINT);
-        c5.setIsKey(true);
-        c5.setIsAllowNull(true);
-        columns.add(c5);
-
-        return columns;
-    }
-
-}
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java
index a4a09d0870..1687d0c1a6 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java
@@ -67,9 +67,9 @@ public class RepeatNodeTest extends TestWithFeService {
         String sql2 = "select /*+ SET_VAR(enable_nereids_planner=false) */ (id 
+ 1) id_, name, sum(cost) from mycost group by grouping sets((id_, name),());";
         String explainString2 = getSQLPlanOrErrorMsg("explain " + sql2);
         System.out.println(explainString2);
-        Assertions.assertTrue(explainString2.contains("exprs: ((`id` + 1)), 
`name`, `cost`"));
+        Assertions.assertTrue(explainString2.contains("exprs: (`id` + 1), 
`name`, `cost`"));
         Assertions.assertTrue(
-                explainString2.contains(" output slots: `((`id` + 1))`, 
``name``, ``cost``, ``GROUPING_ID``"));
+                explainString2.contains(" output slots: `(`id` + 1)`, 
``name``, ``cost``, ``GROUPING_ID``"));
 
         String sql3 = "select /*+ SET_VAR(enable_nereids_planner=false) */ 1 
as id_, name, sum(cost) from mycost group by grouping sets((id_, name),());";
         String explainString3 = getSQLPlanOrErrorMsg("explain " + sql3);
diff --git 
a/regression-test/data/datatype_p0/agg_state/max/test_agg_state_max.out 
b/regression-test/data/datatype_p0/agg_state/max/test_agg_state_max.out
index ea6bed327d..3f5d60ed72 100644
--- a/regression-test/data/datatype_p0/agg_state/max/test_agg_state_max.out
+++ b/regression-test/data/datatype_p0/agg_state/max/test_agg_state_max.out
@@ -9,22 +9,6 @@
 6      6999
 7      7999
 
--- !select --
-7999
-
--- !select --
-0      999
-1      1999
-2      2999
-3      3999
-4      4999
-5      5999
-6      6999
-7      7999
-
--- !select --
-7999
-
 -- !select --
 0      999
 1      1999
@@ -36,20 +20,3 @@
 7      7999
 100    \N
 
--- !select --
-7999
-
--- !select --
-0      999
-1      1999
-2      2999
-3      3999
-4      4999
-5      5999
-6      6999
-7      7999
-100    \N
-
--- !select --
-7999
-
diff --git a/regression-test/data/mv_p0/k1ap2spa/k1ap2spa.out 
b/regression-test/data/mv_p0/k1ap2spa/k1ap2spa.out
index c7d574c013..16fe69c3ff 100644
--- a/regression-test/data/mv_p0/k1ap2spa/k1ap2spa.out
+++ b/regression-test/data/mv_p0/k1ap2spa/k1ap2spa.out
@@ -1,13 +1,11 @@
 -- This file is automatically generated. You should know what you did if you 
want to edit this
 -- !select_star --
-\N     4       \N      d
 -4     -4      -4      d
 1      1       1       a
 2      2       2       b
 3      -3      \N      c
 
 -- !select_mv --
-\N     5
 2      2
 3      3
 4      2
diff --git 
a/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy 
b/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy
index bce225a7cd..c022b829e9 100644
--- a/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy
+++ b/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy
@@ -16,6 +16,9 @@
 // under the License.
 
 suite("test_agg_state_max") {
+    // todo: will core dump now, need fix.
+    sql"set enable_nereids_planner=false;"
+    
     sql """ DROP TABLE IF EXISTS a_table; """
     sql """
             create table a_table(
@@ -36,24 +39,8 @@ suite("test_agg_state_max") {
             select e1/1000,max_state(e1) from 
                 (select 1 k1) as t lateral view explode_numbers(8000) tmp1 as 
e1;"""
 
-    sql"set enable_nereids_planner=false;"
-    qt_select """ select k1,max_merge(k2) from a_table group by k1 order by k1;
-             """
-    qt_select """ select max_merge(tmp) from (select k1,max_union(k2) tmp from 
a_table group by k1)t;
-             """
-    test {
-        sql "select k1,min_merge(k2) from a_table group by k1 order by k1;"
-        exception "not match function"
-    }
-    sql"set enable_nereids_planner=true;"
     qt_select """ select k1,max_merge(k2) from a_table group by k1 order by k1;
              """
-    qt_select """ select max_merge(tmp) from (select k1,max_union(k2) tmp from 
a_table group by k1)t;
-             """
-    test {
-        sql "select k1,min_merge(k2) from a_table group by k1 order by k1;"
-        exception "not match function"
-    }
 
     sql """ DROP TABLE IF EXISTS a_table2; """
     sql """
@@ -70,14 +57,6 @@ suite("test_agg_state_max") {
             select e1/1000,max_state(e1) from 
                 (select 1 k1) as t lateral view explode_numbers(8000) tmp1 as 
e1;"""
 
-    sql"set enable_nereids_planner=false;"
-    qt_select """ select k1,max_merge(k2) from a_table2 group by k1 order by 
k1;
-             """
-    qt_select """ select max_merge(tmp) from (select k1,max_union(k2) tmp from 
a_table group by k1)t;
-             """
-    sql"set enable_nereids_planner=true;"
     qt_select """ select k1,max_merge(k2) from a_table2 group by k1 order by 
k1;
              """
-    qt_select """ select max_merge(tmp) from (select k1,max_union(k2) tmp from 
a_table group by k1)t;
-            """
 }
diff --git a/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy 
b/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy
index 3bfbca6d2b..24d250ce97 100644
--- a/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy
+++ b/regression-test/suites/mv_p0/k1ap2spa/k1ap2spa.groovy
@@ -40,7 +40,6 @@ suite ("k1ap2spa") {
     createMV("create materialized view k1ap2spa as select 
abs(k1)+1,sum(abs(k2+1)) from d_table group by abs(k1)+1;")
 
     sql "insert into d_table select -4,-4,-4,'d';"
-    sql "insert into d_table(k4,k2) values('d',4);"
 
     qt_select_star "select * from d_table order by k1;"
 


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to