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]