diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index e60f34b62840439..a162a0c48781e34 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -3285,6 +3285,14 @@ public static int metaServiceRpcRetryTimes() {
@ConfField(mutable = true, description = {"存算分离模式下commit阶段等锁超时时间,默认5s"})
public static int try_commit_lock_timeout_seconds = 5;
+ @ConfField(mutable = true, description = {"是否在事务提交时对所有表启用提交锁。设置为 true 时,所有表都会使用提交锁。"
+ + "设置为 false 时,仅对 Merge-On-Write 表使用提交锁。默认值为 true。",
+ "Whether to enable commit lock for all tables during transaction commit."
+ + "If true, commit lock will be applied to all tables."
+ + "If false, commit lock will only be applied to Merge-On-Write tables."
+ + "Default value is true." })
+ public static boolean enable_commit_lock_for_all_tables = true;
+
@ConfField(mutable = true, description = {"存算分离模式下是否开启大事务提交,默认false"})
public static boolean enable_cloud_txn_lazy_commit = false;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java
index 0084e677d21da97..e1e722443e40f48 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/CloudGlobalTransactionMgr.java
@@ -1166,7 +1166,21 @@ private void commitTransactionWithSubTxns(long dbId, List
tableList, long
executeCommitTxnRequest(commitTxnRequest, transactionId, false, null);
}
- // add some log and get commit lock, mainly used for mow tables
+ private List getTablesNeedCommitLock(List tableList) {
+ if (Config.enable_commit_lock_for_all_tables) {
+ // If enabled, lock all tables
+ return tableList.stream()
+ .sorted(Comparator.comparingLong(Table::getId))
+ .collect(Collectors.toList());
+ } else {
+ // If disabled, only lock MOW tables
+ return tableList.stream()
+ .filter(table -> table instanceof OlapTable && ((OlapTable) table).getEnableUniqueKeyMergeOnWrite())
+ .sorted(Comparator.comparingLong(Table::getId))
+ .collect(Collectors.toList());
+ }
+ }
+
private void beforeCommitTransaction(List tableList, long transactionId, long timeoutMillis)
throws UserException {
for (int i = 0; i < tableList.size(); i++) {
@@ -1180,29 +1194,21 @@ private void beforeCommitTransaction(List tableList, long transactionId,
}
}
- // Get tables that require commit lock - only MOW tables need this:
- // 1. Filter to keep only OlapTables with MOW enabled
- // 2. Sort by table ID to maintain consistent locking order and prevent deadlocks
- List mowTableList = tableList.stream()
- .filter(table -> table instanceof OlapTable && ((OlapTable) table).getEnableUniqueKeyMergeOnWrite())
- .sorted(Comparator.comparingLong(Table::getId))
- .collect(Collectors.toList());
- increaseWaitingLockCount(mowTableList);
- if (!MetaLockUtils.tryCommitLockTables(mowTableList, timeoutMillis, TimeUnit.MILLISECONDS)) {
- decreaseWaitingLockCount(mowTableList);
+ List tablesToLock = getTablesNeedCommitLock(tableList);
+ increaseWaitingLockCount(tablesToLock);
+ if (!MetaLockUtils.tryCommitLockTables(tablesToLock, timeoutMillis, TimeUnit.MILLISECONDS)) {
+ decreaseWaitingLockCount(tablesToLock);
// DELETE_BITMAP_LOCK_ERR will be retried on be
throw new UserException(InternalErrorCode.DELETE_BITMAP_LOCK_ERR,
"get table cloud commit lock timeout, tableList=("
- + StringUtils.join(mowTableList, ",") + ")");
+ + StringUtils.join(tablesToLock, ",") + ")");
}
}
private void afterCommitTransaction(List tableList) {
- List mowTableList = tableList.stream()
- .filter(table -> table instanceof OlapTable && ((OlapTable) table).getEnableUniqueKeyMergeOnWrite())
- .collect(Collectors.toList());
- decreaseWaitingLockCount(mowTableList);
- MetaLockUtils.commitUnlockTables(mowTableList);
+ List tablesToUnlock = getTablesNeedCommitLock(tableList);
+ decreaseWaitingLockCount(tablesToUnlock);
+ MetaLockUtils.commitUnlockTables(tablesToUnlock);
}
@Override
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 3d2489edd7ea994..bd438f064dd9d52 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
@@ -957,7 +957,13 @@ public PlanFragment visitPhysicalHashAggregate(
// 1. generate slot reference for each group expression
List groupSlots = collectGroupBySlots(groupByExpressions, outputExpressions);
ArrayList execGroupingExpressions = groupByExpressions.stream()
- .map(e -> ExpressionTranslator.translate(e, context))
+ .map(e -> {
+ Expr result = ExpressionTranslator.translate(e, context);
+ if (result == null) {
+ throw new RuntimeException("translate " + e + " failed");
+ }
+ return result;
+ })
.collect(Collectors.toCollection(ArrayList::new));
// 2. collect agg expressions and generate agg function to slot reference map
List aggFunctionOutput = Lists.newArrayList();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java
index a8038ab30b04aec..d7a13148c1040d4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java
@@ -46,6 +46,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
+import java.util.stream.Collectors;
/**
* create project under aggregate to enable CSE
@@ -102,8 +103,38 @@ public Plan visitPhysicalHashAggregate(PhysicalHashAggregate extends Plan> agg
}
if (aggregate.child() instanceof PhysicalProject) {
+ List newProjections = Lists.newArrayList();
+ // do column prune
+ // case 1:
+ // original plan
+ // agg(groupKey[C+1, abs(C+1)]
+ // -->project(A+B as C)
+ //
+ // "A+B as C" should be reserved
+ // new plan
+ // agg(groupKey=[D, abs(D)])
+ // -->project(A+B as C, C+1 as D)
+ // case 2:
+ // original plan
+ // agg(groupKey[A+1, abs(A+1)], output[sum(B)])
+ // --> project(A, B)
+ // "A+1" is extracted, we have
+ // plan1:
+ // agg(groupKey[X, abs(X)], output[sum(B)])
+ // --> project(A, B, A+1 as X)
+ // then column prune(A should be pruned, because it is not used directly by AGG)
+ // we have plan2:
+ // agg(groupKey[X, abs(X)], output[sum(B)])
+ // -->project(B, A+1 as X)
PhysicalProject extends Plan> project = (PhysicalProject extends Plan>) aggregate.child();
- List newProjections = Lists.newArrayList(project.getProjects());
+ Set newInputSlots = aggOutputReplaced.stream()
+ .flatMap(expr -> expr.getInputSlots().stream())
+ .collect(Collectors.toSet());
+ for (NamedExpression expr : project.getProjects()) {
+ if (!(expr instanceof SlotReference) || newInputSlots.contains(expr)) {
+ newProjections.add(expr);
+ }
+ }
newProjections.addAll(cseCandidates.values());
project = project.withProjectionsAndChild(newProjections, (Plan) project.child());
aggregate = (PhysicalHashAggregate extends Plan>) aggregate
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java
index bcd12ac17d2579b..e86727a25cde76a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java
@@ -107,7 +107,10 @@
import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughSort;
import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughWindow;
import org.apache.doris.nereids.rules.rewrite.PushDownJoinOtherCondition;
+import org.apache.doris.nereids.rules.rewrite.PushDownLimitDistinctThroughJoin;
import org.apache.doris.nereids.rules.rewrite.PushDownProjectThroughLimit;
+import org.apache.doris.nereids.rules.rewrite.PushDownTopNDistinctThroughJoin;
+import org.apache.doris.nereids.rules.rewrite.PushDownTopNThroughJoin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
@@ -132,6 +135,9 @@ public class RuleSet {
.add(PushDownProjectThroughInnerOuterJoin.INSTANCE)
.add(PushDownProjectThroughSemiJoin.INSTANCE)
.add(TransposeAggSemiJoinProject.INSTANCE)
+ .addAll(new PushDownTopNThroughJoin().buildRules())
+ .addAll(new PushDownLimitDistinctThroughJoin().buildRules())
+ .addAll(new PushDownTopNDistinctThroughJoin().buildRules())
.build();
public static final List PUSH_DOWN_FILTERS = ImmutableList.of(
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
index 1e76307327d22cf..82aac806bfbde7d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionOptimization.java
@@ -29,6 +29,7 @@
import org.apache.doris.nereids.rules.expression.rules.SimplifyComparisonPredicate;
import org.apache.doris.nereids.rules.expression.rules.SimplifyInPredicate;
import org.apache.doris.nereids.rules.expression.rules.SimplifyRange;
+import org.apache.doris.nereids.rules.expression.rules.SimplifySelfComparison;
import org.apache.doris.nereids.rules.expression.rules.TopnToMax;
import com.google.common.collect.ImmutableList;
@@ -46,6 +47,7 @@ public class ExpressionOptimization extends ExpressionRewrite {
SimplifyComparisonPredicate.INSTANCE,
SimplifyInPredicate.INSTANCE,
SimplifyRange.INSTANCE,
+ SimplifySelfComparison.INSTANCE,
DateFunctionRewrite.INSTANCE,
ArrayContainToArrayOverlap.INSTANCE,
CaseWhenToIf.INSTANCE,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
index 7f83ab8a090fd83..439be53dcbcfa6a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRuleType.java
@@ -51,6 +51,7 @@ public enum ExpressionRuleType {
SIMPLIFY_IN_PREDICATE,
SIMPLIFY_NOT_EXPR,
SIMPLIFY_RANGE,
+ SIMPLIFY_SELF_COMPARISON,
SUPPORT_JAVA_DATE_FORMATTER,
TOPN_TO_MAX;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifySelfComparison.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifySelfComparison.java
new file mode 100644
index 000000000000000..dbd89964710de5a
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/SimplifySelfComparison.java
@@ -0,0 +1,74 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.expression.rules;
+
+import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher;
+import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory;
+import org.apache.doris.nereids.rules.expression.ExpressionRuleType;
+import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
+import org.apache.doris.nereids.trees.expressions.EqualTo;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.GreaterThan;
+import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
+import org.apache.doris.nereids.trees.expressions.LessThan;
+import org.apache.doris.nereids.trees.expressions.LessThanEqual;
+import org.apache.doris.nereids.trees.expressions.NullSafeEqual;
+import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * simplify with self comparison
+ * such as: a = a --> TRUE
+ * a + b > a + b --> FALSE
+ */
+public class SimplifySelfComparison implements ExpressionPatternRuleFactory {
+ public static SimplifySelfComparison INSTANCE = new SimplifySelfComparison();
+
+ @Override
+ public List> buildRules() {
+ return ImmutableList.of(
+ matchesType(ComparisonPredicate.class)
+ .then(this::rewrite)
+ .toRule(ExpressionRuleType.SIMPLIFY_SELF_COMPARISON)
+ );
+ }
+
+ private Expression rewrite(ComparisonPredicate comparison) {
+ Expression left = comparison.left();
+ if (left.equals(comparison.right())) {
+ if (comparison instanceof EqualTo
+ || comparison instanceof GreaterThanEqual
+ || comparison instanceof LessThanEqual) {
+ return ExpressionUtils.trueOrNull(left);
+ }
+ if (comparison instanceof NullSafeEqual) {
+ return BooleanLiteral.TRUE;
+ }
+ if (comparison instanceof GreaterThan || comparison instanceof LessThan) {
+ return ExpressionUtils.falseOrNull(left);
+ }
+ }
+
+ return comparison;
+ }
+
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoin.java
index 21f777204b2a0ea..590e20a776366aa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownLimitDistinctThroughJoin.java
@@ -25,6 +25,7 @@
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalLimit;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+import org.apache.doris.qe.ConnectContext;
import com.google.common.collect.ImmutableList;
@@ -42,6 +43,10 @@ public List buildRules() {
// limit -> distinct -> join
logicalLimit(logicalAggregate(logicalJoin())
.when(LogicalAggregate::isDistinct))
+ .when(limit ->
+ ConnectContext.get() != null
+ && ConnectContext.get().getSessionVariable().topnOptLimitThreshold
+ >= limit.getLimit() + limit.getOffset())
.then(limit -> {
LogicalAggregate> agg = limit.child();
LogicalJoin join = agg.child();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNDistinctThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNDistinctThroughJoin.java
index f2dde7ba2a8be52..6c280146c8ab2ca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNDistinctThroughJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNDistinctThroughJoin.java
@@ -27,6 +27,7 @@
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.trees.plans.logical.LogicalTopN;
import org.apache.doris.nereids.util.PlanUtils;
+import org.apache.doris.qe.ConnectContext;
import com.google.common.collect.ImmutableList;
@@ -45,6 +46,10 @@ public List buildRules() {
// topN -> join
logicalTopN(logicalAggregate(logicalJoin()).when(LogicalAggregate::isDistinct))
// TODO: complex order by
+ .when(topn ->
+ ConnectContext.get() != null
+ && ConnectContext.get().getSessionVariable().topnOptLimitThreshold
+ >= topn.getLimit() + topn.getOffset())
.when(topN -> topN.getOrderKeys().stream().map(OrderKey::getExpr)
.allMatch(Slot.class::isInstance))
.then(topN -> {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNThroughJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNThroughJoin.java
index 28a7f2688be8cd2..4d15dae59e76dac 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNThroughJoin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownTopNThroughJoin.java
@@ -25,6 +25,7 @@
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.trees.plans.logical.LogicalTopN;
+import org.apache.doris.qe.ConnectContext;
import com.google.common.collect.ImmutableList;
@@ -43,6 +44,10 @@ public List buildRules() {
// topN -> join
logicalTopN(logicalJoin())
// TODO: complex orderby
+ .when(topn ->
+ ConnectContext.get() != null
+ && ConnectContext.get().getSessionVariable().topnOptLimitThreshold
+ >= topn.getLimit() + topn.getOffset())
.when(topN -> topN.getOrderKeys().stream().map(OrderKey::getExpr)
.allMatch(Slot.class::isInstance))
.then(topN -> {
@@ -102,7 +107,6 @@ private Plan pushLimitThroughJoin(LogicalTopN extends Plan> topN, LogicalJoin<
}
return null;
case CROSS_JOIN:
-
if (join.left().getOutputSet().containsAll(orderbySlots)) {
return join.withChildren(
topN.withLimitChild(topN.getLimit() + topN.getOffset(), 0, join.left()),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
index 64019818f5d1e74..5daf3bb254fc24c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java
@@ -501,8 +501,8 @@ private Statistics computeOlapScan(OlapScan olapScan) {
// mv is selected, return its estimated stats
Optional optStats = cascadesContext.getStatementContext()
.getStatistics(((Relation) olapScan).getRelationId());
- LOG.info("computeOlapScan optStats isPresent {}, tableRowCount is {}",
- optStats.isPresent(), tableRowCount);
+ LOG.info("computeOlapScan optStats isPresent {}, tableRowCount is {}, table name is {}",
+ optStats.isPresent(), tableRowCount, olapTable.getQualifiedName());
if (optStats.isPresent()) {
double selectedPartitionsRowCount = getSelectedPartitionRowCount(olapScan, tableRowCount);
LOG.info("computeOlapScan optStats is {}, selectedPartitionsRowCount is {}", optStats.get(),
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifySelfComparisonTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifySelfComparisonTest.java
new file mode 100644
index 000000000000000..be59404b46786b4
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/rules/SimplifySelfComparisonTest.java
@@ -0,0 +1,43 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.expression.rules;
+
+import org.apache.doris.nereids.rules.expression.ExpressionRewrite;
+import org.apache.doris.nereids.rules.expression.ExpressionRewriteTestHelper;
+import org.apache.doris.nereids.rules.expression.ExpressionRuleExecutor;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.jupiter.api.Test;
+
+class SimplifySelfComparisonTest extends ExpressionRewriteTestHelper {
+
+ @Test
+ void testRewrite() {
+ executor = new ExpressionRuleExecutor(ImmutableList.of(
+ ExpressionRewrite.bottomUp(SimplifySelfComparison.INSTANCE)
+ ));
+
+ assertRewriteAfterTypeCoercion("TA + TB = TA + TB", "NOT ((TA + TB) IS NULL) OR NULL");
+ assertRewriteAfterTypeCoercion("TA + TB >= TA + TB", "NOT ((TA + TB) IS NULL) OR NULL");
+ assertRewriteAfterTypeCoercion("TA + TB <= TA + TB", "NOT ((TA + TB) IS NULL) OR NULL");
+ assertRewriteAfterTypeCoercion("TA + TB <=> TA + TB", "TRUE");
+ assertRewriteAfterTypeCoercion("TA + TB > TA + TB", "(TA + TB) IS NULL AND NULL");
+ assertRewriteAfterTypeCoercion("TA + TB < TA + TB", "(TA + TB) IS NULL AND NULL");
+ }
+
+}
diff --git a/regression-test/data/mv_p0/unique/unique_rewrite.out b/regression-test/data/mv_p0/unique_rewrite/unique_rewrite.out
similarity index 100%
rename from regression-test/data/mv_p0/unique/unique_rewrite.out
rename to regression-test/data/mv_p0/unique_rewrite/unique_rewrite.out
diff --git a/regression-test/data/nereids_p0/hint/fix_leading.out b/regression-test/data/nereids_p0/hint/fix_leading.out
index 372ffad30a3b9bb..1b8ddec73a35a8c 100644
--- a/regression-test/data/nereids_p0/hint/fix_leading.out
+++ b/regression-test/data/nereids_p0/hint/fix_leading.out
@@ -1,10 +1,11 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !select1 --
+-- explain shape plan select /*+ leading({t1 t2}{t3 t4}) */ * from t1 join t2 on c2 = c2 join t3 on c1 = c3 join t4 on c1 = c4;
PhysicalResultSink
--hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t3.c3) and (t1.c1 = t4.c4)) otherCondition=()
----NestedLoopJoin[CROSS_JOIN]
------PhysicalOlapScan[t1]
-------filter((t2.c2 = t2.c2))
+------filter(OR[( not c2 IS NULL),NULL])
--------PhysicalOlapScan[t2]
----hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=()
------PhysicalOlapScan[t3]
@@ -16,219 +17,291 @@ UnUsed:
SyntaxError:
-- !select2_1_1 --
+-- select count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_2 --
+-- select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_3 --
+-- select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_4 --
+-- select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_5 --
+-- select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_6 --
+-- select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_7 --
+-- select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_8 --
+-- select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_9 --
+-- select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_10 --
+-- select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_11 --
+-- select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_12 --
+-- select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_1_13 --
+-- select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 left join t3 on c2 = c3;
1876000
-- !select2_2_1 --
+-- select count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_2 --
+-- select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_3 --
+-- select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_4 --
+-- select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_5 --
+-- select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_6 --
+-- select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_7 --
+-- select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_8 --
+-- select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_9 --
+-- select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_10 --
+-- select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_11 --
+-- select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_12 --
+-- select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_2_13 --
+-- select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 right join t3 on c2 = c3;
1751202
-- !select2_3_1 --
+-- select count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_2 --
+-- select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_3 --
+-- select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_4 --
+-- select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_5 --
+-- select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_6 --
+-- select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_7 --
+-- select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_8 --
+-- select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_9 --
+-- select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_10 --
+-- select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_11 --
+-- select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_12 --
+-- select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_3_13 --
+-- select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_4_1 --
+-- select count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_2 --
+-- select /*+ leading(t1 t2 t3) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_3 --
+-- select /*+ leading(t1 {t2 t3}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_4 --
+-- select /*+ leading(t1 t3 t2) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_5 --
+-- select /*+ leading(t1 {t2 t2}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_6 --
+-- select /*+ leading(t2 t1 t3) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_7 --
+-- select /*+ leading(t2 {t1 t3}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_8 --
+-- select /*+ leading(t2 t3 t1) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_9 --
+-- select /*+ leading(t2 {t3 t1}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_10 --
+-- select /*+ leading(t3 t1 t2) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_11 --
+-- select /*+ leading(t3 {t1 t2}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_12 --
+-- select /*+ leading(t3 t2 t1) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_4_13 --
+-- select /*+ leading(t3 {t2 t1}) */ count(*) from t1 left join t2 on c2 = c2 left anti join t3 on c2 = c3;
125000
-- !select2_5_1 --
+-- select count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_2 --
+-- select /*+ leading(t1 t2 t3) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_3 --
+-- select /*+ leading(t1 {t2 t3}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_4 --
+-- select /*+ leading(t1 t3 t2) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_5 --
+-- select /*+ leading(t1 {t2 t2}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_6 --
+-- select /*+ leading(t2 t1 t3) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_7 --
+-- select /*+ leading(t2 {t1 t3}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_8 --
+-- select /*+ leading(t2 t3 t1) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_9 --
+-- select /*+ leading(t2 {t3 t1}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_10 --
+-- select /*+ leading(t3 t1 t2) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_11 --
+-- select /*+ leading(t3 {t1 t2}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_12 --
+-- select /*+ leading(t3 t2 t1) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select2_5_13 --
+-- select /*+ leading(t3 {t2 t1}) */ count(*) from t1 right join t2 on c2 = c2 left semi join t3 on c2 = c3;
816000
-- !select3_1 --
+-- select /*+ leading(t1) */ count(*) from t1 join t2 on c1 = c2;
1719
-- !select3_2 --
+-- select /*+ leading({t1}) */ count(*) from t1 join t2 on c1 = c2;
1719
-- !select3_3 --
+-- select /*+ leading(t1 {t2}) */ count(*) from t1 join t2 on c1 = c2;
1719
-- !select3_4 --
+-- with cte as (select c1 from t1) select count(*) from t1 join (select /*+ leading(cte t2) */ c2 from t2 join cte on c2 = cte.c1) as alias on t1.c1 = alias.c2;
5169
-- !select4_1 --
+-- select count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;
1000
-- !select4_2 --
+-- select /*+ leading(t1 t2 t3)*/ count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;
1000
-- !select4_3 --
+-- explain shape plan select /*+ leading(t1 t2 t3)*/ count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;
PhysicalResultSink
--hashAgg[GLOBAL]
----hashAgg[LOCAL]
@@ -245,6 +318,7 @@ UnUsed:
SyntaxError:
-- !select6_1 --
+-- explain shape plan select /*+ leading(t1 {{t2 t3}{t4 t5}} t6) */ count(*) from t1 join t2 on c1 = c2 join t3 on c1 = c3 join t4 on c1 = c4 join t5 on c1 = c5 join t6 on c1 = c6;
PhysicalResultSink
--hashAgg[GLOBAL]
----hashAgg[LOCAL]
diff --git a/regression-test/data/nereids_syntax_p0/mv/newMv/multi_slot4.out b/regression-test/data/nereids_syntax_p0/mv/newMv/multi_slot4.out
index 5b5000679863664..264a653fd3a3d04 100644
--- a/regression-test/data/nereids_syntax_p0/mv/newMv/multi_slot4.out
+++ b/regression-test/data/nereids_syntax_p0/mv/newMv/multi_slot4.out
@@ -1,14 +1,22 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !select_star --
-4 -4 -4 d
+-4 -4 -4 d
+-4 -4 -4 d
+1 1 1 a
+1 1 1 a
1 1 1 a
2 2 2 b
+2 2 2 b
+2 2 2 b
+3 -3 \N c
+3 -3 \N c
3 -3 \N c
3 -3 \N c
-- !select_mv --
--3 1
-2 7
-3 9
+-3 3
+2 21
+3 27
4 \N
diff --git a/regression-test/data/query_p0/show/test_show_create_materialized_view.out b/regression-test/data/query_p0/show/test_show_create_materialized_view.out
index 040ebf56f29fc52..1d874bb14d3c5a9 100644
--- a/regression-test/data/query_p0/show/test_show_create_materialized_view.out
+++ b/regression-test/data/query_p0/show/test_show_create_materialized_view.out
@@ -1,4 +1,4 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !cmd --
-table_for_mv_test mv_show_create_materialized_view CREATE MATERIALIZED VIEW mv_show_create_materialized_view AS\n SELECT id, name, SUM(value) AS total_value\n FROM table_for_mv_test\n GROUP BY id, name;\n
+table_for_mv_test mv_show_create_materialized_view \n CREATE MATERIALIZED VIEW mv_show_create_materialized_view \n AS \n SELECT id, name, SUM(value) AS total_value\n FROM table_for_mv_test\n GROUP BY id, name\n
diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
index 5cbed97829c990c..155cb5868bce598 100644
--- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
+++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
@@ -742,10 +742,16 @@ class Suite implements GroovyInterceptable {
return result
}
+ // Should use create_sync_mv, this method only check the sync mv in current db
+ // If has multi sync mv in db, may make mistake
+ @Deprecated
void createMV(String sql) {
(new CreateMVAction(context, sql)).run()
}
+ // Should use create_sync_mv, this method only check the sync mv in current db
+ // If has multi sync mv in db, may make mistake
+ @Deprecated
void createMV(String sql, String expection) {
(new CreateMVAction(context, sql, expection)).run()
}
@@ -1475,80 +1481,101 @@ class Suite implements GroovyInterceptable {
return debugPoint
}
- void waitingMTMVTaskFinishedByMvName(String mvName) {
+ def waitingMTMVTaskFinishedByMvName = { mvName, dbName = context.dbName ->
Thread.sleep(2000);
- String showTasks = "select TaskId,JobId,JobName,MvId,Status,MvName,MvDatabaseName,ErrorMsg from tasks('type'='mv') where MvName = '${mvName}' order by CreateTime ASC"
+ String showTasks = "select TaskId,JobId,JobName,MvId,Status,MvName,MvDatabaseName,ErrorMsg from tasks('type'='mv') where MvDatabaseName = '${dbName}' and MvName = '${mvName}' order by CreateTime DESC LIMIT 1"
String status = "NULL"
List> result
long startTime = System.currentTimeMillis()
long timeoutTimestamp = startTime + 5 * 60 * 1000 // 5 min
- do {
+ List toCheckTaskRow = new ArrayList<>();
+ while (timeoutTimestamp > System.currentTimeMillis() && (status != "SUCCESS")) {
result = sql(showTasks)
- logger.info("result: " + result.toString())
- if (!result.isEmpty()) {
- status = result.last().get(4)
- }
+ logger.info("current db is " + dbName + ", showTasks is " + showTasks)
+ if (result.isEmpty()) {
+ logger.info("waitingMTMVTaskFinishedByMvName toCheckTaskRow is empty")
+ Thread.sleep(1000);
+ continue;
+ }
+ toCheckTaskRow = result.get(0);
+ status = toCheckTaskRow.get(4)
logger.info("The state of ${showTasks} is ${status}")
Thread.sleep(1000);
- } while (timeoutTimestamp > System.currentTimeMillis() && (status == 'PENDING' || status == 'RUNNING' || status == 'NULL'))
+ }
if (status != "SUCCESS") {
logger.info("status is not success")
}
Assert.assertEquals("SUCCESS", status)
def show_tables = sql """
- show tables from ${result.last().get(6)};
+ show tables from ${toCheckTaskRow.get(6)};
"""
- def db_id = getDbId(result.last().get(6))
- def table_id = getTableId(result.last().get(6), mvName)
+ def db_id = getDbId(toCheckTaskRow.get(6))
+ def table_id = getTableId(toCheckTaskRow.get(6), mvName)
logger.info("waitingMTMVTaskFinished analyze mv name is " + mvName
- + ", db name is " + result.last().get(6)
+ + ", db name is " + toCheckTaskRow.get(6)
+ ", show_tables are " + show_tables
+ ", db_id is " + db_id
+ ", table_id " + table_id)
- sql "analyze table ${result.last().get(6)}.${mvName} with sync;"
+ sql "analyze table ${toCheckTaskRow.get(6)}.${mvName} with sync;"
}
- void waitingMTMVTaskFinishedByMvNameAllowCancel(String mvName) {
+ def waitingMTMVTaskFinishedByMvNameAllowCancel = {mvName, dbName = context.dbName ->
Thread.sleep(2000);
- String showTasks = "select TaskId,JobId,JobName,MvId,Status,MvName,MvDatabaseName,ErrorMsg from tasks('type'='mv') where MvName = '${mvName}' order by CreateTime ASC"
+ String showTasks = "select TaskId,JobId,JobName,MvId,Status,MvName,MvDatabaseName,ErrorMsg from tasks('type'='mv') where MvDatabaseName = '${dbName}' and MvName = '${mvName}' order by CreateTime DESC LIMIT 1"
+
String status = "NULL"
List> result
long startTime = System.currentTimeMillis()
long timeoutTimestamp = startTime + 5 * 60 * 1000 // 5 min
- do {
+ List toCheckTaskRow = new ArrayList<>();
+ while (timeoutTimestamp > System.currentTimeMillis() && (status != "SUCCESS")) {
result = sql(showTasks)
- logger.info("result: " + result.toString())
- if (!result.isEmpty()) {
- status = result.last().get(4)
- }
+ logger.info("current db is " + dbName + ", showTasks result: " + result.toString())
+ if (result.isEmpty()) {
+ logger.info("waitingMTMVTaskFinishedByMvName toCheckTaskRow is empty")
+ Thread.sleep(1000);
+ continue;
+ }
+ toCheckTaskRow = result.get(0)
+ status = toCheckTaskRow.get(4)
logger.info("The state of ${showTasks} is ${status}")
Thread.sleep(1000);
- } while (timeoutTimestamp > System.currentTimeMillis() && (status == 'PENDING' || status == 'RUNNING' || status == 'NULL' || status == 'CANCELED'))
+ }
if (status != "SUCCESS") {
logger.info("status is not success")
assertTrue(result.toString().contains("same table"))
}
// Need to analyze materialized view for cbo to choose the materialized view accurately
- logger.info("waitingMTMVTaskFinished analyze mv name is " + result.last().get(5))
- sql "analyze table ${result.last().get(6)}.${mvName} with sync;"
+ logger.info("waitingMTMVTaskFinished analyze mv name is " + toCheckTaskRow.get(5))
+ sql "analyze table ${toCheckTaskRow.get(6)}.${mvName} with sync;"
}
- void waitingMVTaskFinishedByMvName(String dbName, String tableName) {
+ void waitingMVTaskFinishedByMvName(String dbName, String tableName, String indexName) {
Thread.sleep(2000)
- String showTasks = "SHOW ALTER TABLE MATERIALIZED VIEW from ${dbName} where TableName='${tableName}' ORDER BY CreateTime ASC"
+ String showTasks = "SHOW ALTER TABLE MATERIALIZED VIEW from ${dbName} where TableName='${tableName}' ORDER BY CreateTime DESC"
String status = "NULL"
List> result
long startTime = System.currentTimeMillis()
long timeoutTimestamp = startTime + 5 * 60 * 1000 // 5 min
- do {
+ List toCheckTaskRow = new ArrayList<>();
+ while (timeoutTimestamp > System.currentTimeMillis() && (status != 'FINISHED')) {
result = sql(showTasks)
- logger.info("result: " + result.toString())
- if (!result.isEmpty()) {
- status = result.last().get(8)
+ logger.info("crrent db is " + dbName + ", showTasks result: " + result.toString())
+ // just consider current db
+ for (List taskRow : result) {
+ if (taskRow.get(5).equals(indexName)) {
+ toCheckTaskRow = taskRow;
+ }
+ }
+ if (toCheckTaskRow.isEmpty()) {
+ logger.info("waitingMVTaskFinishedByMvName toCheckTaskRow is empty")
+ Thread.sleep(1000);
+ continue;
}
+ status = toCheckTaskRow.get(8)
logger.info("The state of ${showTasks} is ${status}")
Thread.sleep(1000);
- } while (timeoutTimestamp > System.currentTimeMillis() && (status != 'FINISHED'))
+ }
if (status != "FINISHED") {
logger.info("status is not success")
}
@@ -1925,6 +1952,15 @@ class Suite implements GroovyInterceptable {
return isReady
}
+ def create_sync_mv = { db, table_name, mv_name, mv_sql ->
+ sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name} ON ${table_name};"""
+ sql"""
+ CREATE MATERIALIZED VIEW ${mv_name}
+ AS ${mv_sql}
+ """
+ waitingMVTaskFinishedByMvName(db, table_name, mv_name)
+ }
+
def create_async_mv = { db, mv_name, mv_sql ->
sql """DROP MATERIALIZED VIEW IF EXISTS ${db}.${mv_name}"""
diff --git a/regression-test/suites/account_p0/test_nereids_authentication.groovy b/regression-test/suites/account_p0/test_nereids_authentication.groovy
index c2ee5f12e8cb1d5..4b8c644495c0396 100644
--- a/regression-test/suites/account_p0/test_nereids_authentication.groovy
+++ b/regression-test/suites/account_p0/test_nereids_authentication.groovy
@@ -49,7 +49,7 @@ suite("test_nereids_authentication", "query") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
def tokens = context.config.jdbcUrl.split('/')
diff --git a/regression-test/suites/account_p0/test_nereids_row_policy.groovy b/regression-test/suites/account_p0/test_nereids_row_policy.groovy
index 38426dc09e6ef42..4463a41c90b4d84 100644
--- a/regression-test/suites/account_p0/test_nereids_row_policy.groovy
+++ b/regression-test/suites/account_p0/test_nereids_row_policy.groovy
@@ -28,7 +28,7 @@ suite("test_nereids_row_policy") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
def assertQueryResult = { size ->
@@ -88,7 +88,7 @@ suite("test_nereids_row_policy") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
dropPolciy "policy0"
diff --git a/regression-test/suites/auth_call/test_account_management_grant_auth.groovy b/regression-test/suites/auth_call/test_account_management_grant_auth.groovy
index eff62e64f88e8a4..a1e1f68916f90a0 100644
--- a/regression-test/suites/auth_call/test_account_management_grant_auth.groovy
+++ b/regression-test/suites/auth_call/test_account_management_grant_auth.groovy
@@ -31,7 +31,7 @@ suite("test_account_management_grant_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_account_management_role_auth.groovy b/regression-test/suites/auth_call/test_account_management_role_auth.groovy
index c3b278100815f81..dab9596b8c2ae5d 100644
--- a/regression-test/suites/auth_call/test_account_management_role_auth.groovy
+++ b/regression-test/suites/auth_call/test_account_management_role_auth.groovy
@@ -32,7 +32,7 @@ suite("test_account_management_role_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_account_management_user_auth.groovy b/regression-test/suites/auth_call/test_account_management_user_auth.groovy
index fe6d6805f47f9bc..9cbb489615b41e6 100644
--- a/regression-test/suites/auth_call/test_account_management_user_auth.groovy
+++ b/regression-test/suites/auth_call/test_account_management_user_auth.groovy
@@ -30,7 +30,7 @@ suite("test_account_management_user_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_assistant_command_auth.groovy b/regression-test/suites/auth_call/test_assistant_command_auth.groovy
index 1b47e566ff31e8a..ba539f129d7bc9a 100644
--- a/regression-test/suites/auth_call/test_assistant_command_auth.groovy
+++ b/regression-test/suites/auth_call/test_assistant_command_auth.groovy
@@ -31,7 +31,7 @@ suite("test_assistant_command_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_cluster_management_auth.groovy b/regression-test/suites/auth_call/test_cluster_management_auth.groovy
index 2061b9dbca773a5..f769f29c967abc9 100644
--- a/regression-test/suites/auth_call/test_cluster_management_auth.groovy
+++ b/regression-test/suites/auth_call/test_cluster_management_auth.groovy
@@ -66,7 +66,7 @@ suite ("test_cluster_management_auth","nonConcurrent,p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_database_management_auth.groovy b/regression-test/suites/auth_call/test_database_management_auth.groovy
index fb643d9ee089e28..71f1902299654b3 100644
--- a/regression-test/suites/auth_call/test_database_management_auth.groovy
+++ b/regression-test/suites/auth_call/test_database_management_auth.groovy
@@ -29,7 +29,7 @@ suite("test_database_management_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
error_in_cloud = "Unsupported"
}
diff --git a/regression-test/suites/auth_call/test_ddl_catalog_auth.groovy b/regression-test/suites/auth_call/test_ddl_catalog_auth.groovy
index 80f71a2e16f94bb..8d6b31d35c92170 100644
--- a/regression-test/suites/auth_call/test_ddl_catalog_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_catalog_auth.groovy
@@ -29,7 +29,7 @@ suite("test_ddl_catalog_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """create catalog if not exists ${catalogNameOther} properties (
diff --git a/regression-test/suites/auth_call/test_ddl_database_auth.groovy b/regression-test/suites/auth_call/test_ddl_database_auth.groovy
index 2821a375ddb5595..bdcf5ff7c39be70 100644
--- a/regression-test/suites/auth_call/test_ddl_database_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_database_auth.groovy
@@ -30,7 +30,7 @@ suite("test_ddl_database_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_encryptkey_auth.groovy b/regression-test/suites/auth_call/test_ddl_encryptkey_auth.groovy
index 0749bdab71780a9..dfa469bfce00288 100644
--- a/regression-test/suites/auth_call/test_ddl_encryptkey_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_encryptkey_auth.groovy
@@ -28,7 +28,7 @@ suite("test_ddl_encryptkey_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_file_auth.groovy b/regression-test/suites/auth_call/test_ddl_file_auth.groovy
index e32c26ecc220d01..35ac9f2632e5cc9 100644
--- a/regression-test/suites/auth_call/test_ddl_file_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_file_auth.groovy
@@ -28,7 +28,7 @@ suite("test_ddl_file_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_function_auth.groovy b/regression-test/suites/auth_call/test_ddl_function_auth.groovy
index 2fa524bf424bd0e..a2e38f0eb6d6c8d 100644
--- a/regression-test/suites/auth_call/test_ddl_function_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_function_auth.groovy
@@ -28,7 +28,7 @@ suite("test_ddl_function_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_index_auth.groovy b/regression-test/suites/auth_call/test_ddl_index_auth.groovy
index 53ba3a0d4b31233..61a727923be5533 100644
--- a/regression-test/suites/auth_call/test_ddl_index_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_index_auth.groovy
@@ -58,7 +58,7 @@ suite("test_ddl_index_auth","p0,auth_call") {
logger.info("cluster:" + clusters)
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """create database ${dbName}"""
sql """
diff --git a/regression-test/suites/auth_call/test_ddl_job_auth.groovy b/regression-test/suites/auth_call/test_ddl_job_auth.groovy
index 45798191e485686..ef75802637757ba 100644
--- a/regression-test/suites/auth_call/test_ddl_job_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_job_auth.groovy
@@ -30,7 +30,7 @@ suite("test_ddl_job_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_mask_view_auth.groovy b/regression-test/suites/auth_call/test_ddl_mask_view_auth.groovy
index 590e75781f19f20..4a8bc2787dc62c1 100644
--- a/regression-test/suites/auth_call/test_ddl_mask_view_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_mask_view_auth.groovy
@@ -29,7 +29,7 @@ suite("test_ddl_mask_view_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_mtmv_auth.groovy b/regression-test/suites/auth_call/test_ddl_mtmv_auth.groovy
index 4db2177ee6ce184..160b6b840b019b3 100644
--- a/regression-test/suites/auth_call/test_ddl_mtmv_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_mtmv_auth.groovy
@@ -30,7 +30,7 @@ suite("test_ddl_mtmv_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_mv_auth.groovy b/regression-test/suites/auth_call/test_ddl_mv_auth.groovy
index 4dbf54fdf0df72e..4b4810604409fe0 100644
--- a/regression-test/suites/auth_call/test_ddl_mv_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_mv_auth.groovy
@@ -30,7 +30,7 @@ suite("test_ddl_mv_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
@@ -80,9 +80,9 @@ suite("test_ddl_mv_auth","p0,auth_call") {
connect(user, "${pwd}", context.config.jdbcUrl) {
sql """use ${dbName}"""
sql """create materialized view ${mvName} as select username from ${dbName}.${tableName};"""
- waitingMVTaskFinishedByMvName(dbName, tableName)
+ waitingMVTaskFinishedByMvName(dbName, tableName, mvName)
sql """alter table ${dbName}.${tableName} add rollup ${rollupName}(username)"""
- waitingMVTaskFinishedByMvName(dbName, tableName)
+ waitingMVTaskFinishedByMvName(dbName, tableName, rollupName)
def mv_res = sql """desc ${dbName}.${tableName} all;"""
logger.info("mv_res: " + mv_res)
diff --git a/regression-test/suites/auth_call/test_ddl_part_table_auth.groovy b/regression-test/suites/auth_call/test_ddl_part_table_auth.groovy
index 34e4766e19e5e2a..5217dc2d0518600 100644
--- a/regression-test/suites/auth_call/test_ddl_part_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_part_table_auth.groovy
@@ -28,7 +28,7 @@ suite("test_ddl_part_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_row_policy_auth.groovy b/regression-test/suites/auth_call/test_ddl_row_policy_auth.groovy
index af1e074f8d7d8c5..f9ac109f7f377f2 100644
--- a/regression-test/suites/auth_call/test_ddl_row_policy_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_row_policy_auth.groovy
@@ -29,7 +29,7 @@ suite("test_ddl_row_policy_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_sql_block_rule_auth.groovy b/regression-test/suites/auth_call/test_ddl_sql_block_rule_auth.groovy
index 3941897a5e9dd57..568ea9723d58c3d 100644
--- a/regression-test/suites/auth_call/test_ddl_sql_block_rule_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_sql_block_rule_auth.groovy
@@ -29,7 +29,7 @@ suite("test_ddl_sql_block_rule_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_table_auth.groovy b/regression-test/suites/auth_call/test_ddl_table_auth.groovy
index c96aeb0d5e2cad3..47ac4e07abc6f98 100644
--- a/regression-test/suites/auth_call/test_ddl_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_table_auth.groovy
@@ -33,7 +33,7 @@ suite("test_ddl_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_ddl_view_auth.groovy b/regression-test/suites/auth_call/test_ddl_view_auth.groovy
index 05f263ada20066e..1a915acdb3cd6f3 100644
--- a/regression-test/suites/auth_call/test_ddl_view_auth.groovy
+++ b/regression-test/suites/auth_call/test_ddl_view_auth.groovy
@@ -29,7 +29,7 @@ suite("test_ddl_view_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_analyze_auth.groovy b/regression-test/suites/auth_call/test_dml_analyze_auth.groovy
index 59706f140e69b15..8bc6a070d615246 100644
--- a/regression-test/suites/auth_call/test_dml_analyze_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_analyze_auth.groovy
@@ -30,7 +30,7 @@ suite("test_dml_analyze_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_broker_load_auth.groovy b/regression-test/suites/auth_call/test_dml_broker_load_auth.groovy
index 3ec26146699bce7..0ea44241bfae2f1 100644
--- a/regression-test/suites/auth_call/test_dml_broker_load_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_broker_load_auth.groovy
@@ -42,7 +42,7 @@ suite("test_dml_broker_load_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy b/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy
index f1aa5d0fa95105f..9bcb95c2830d9f5 100644
--- a/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_cancel_profile_auth.groovy
@@ -28,7 +28,7 @@ suite("test_dml_cancel_profile_auth","p0,auth_call,nonConcurrent") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy b/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy
index d325250f2ce6966..aec3ee4a9947e5b 100644
--- a/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_delete_table_auth.groovy
@@ -30,7 +30,7 @@ suite("test_dml_delete_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_export_table_auth.groovy b/regression-test/suites/auth_call/test_dml_export_table_auth.groovy
index 12812fdf0dde095..ee5674c940a35cc 100644
--- a/regression-test/suites/auth_call/test_dml_export_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_export_table_auth.groovy
@@ -36,7 +36,7 @@ suite("test_dml_export_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_insert_auth.groovy b/regression-test/suites/auth_call/test_dml_insert_auth.groovy
index 6cfe66cb10c6d5b..5b8a20e18f9f702 100644
--- a/regression-test/suites/auth_call/test_dml_insert_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_insert_auth.groovy
@@ -31,7 +31,7 @@ suite("test_dml_insert_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_multi_routine_load_auth.groovy b/regression-test/suites/auth_call/test_dml_multi_routine_load_auth.groovy
index 11fd6c43db46021..5b8ecef7e57eae7 100644
--- a/regression-test/suites/auth_call/test_dml_multi_routine_load_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_multi_routine_load_auth.groovy
@@ -34,7 +34,7 @@ suite("test_dml_multi_routine_load_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_mysql_load_auth.groovy b/regression-test/suites/auth_call/test_dml_mysql_load_auth.groovy
index 3963fe9433ef450..97151a0d001e855 100644
--- a/regression-test/suites/auth_call/test_dml_mysql_load_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_mysql_load_auth.groovy
@@ -30,7 +30,7 @@ suite("test_dml_mysql_load_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_outfile_auth.groovy b/regression-test/suites/auth_call/test_dml_outfile_auth.groovy
index 7edf476883a93bc..5f4096344b86248 100644
--- a/regression-test/suites/auth_call/test_dml_outfile_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_outfile_auth.groovy
@@ -34,7 +34,7 @@ suite("test_dml_outfile_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_routine_load_auth.groovy b/regression-test/suites/auth_call/test_dml_routine_load_auth.groovy
index db6698b01af1948..025ac555356aa70 100644
--- a/regression-test/suites/auth_call/test_dml_routine_load_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_routine_load_auth.groovy
@@ -33,7 +33,7 @@ suite("test_dml_routine_load_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_select_udf_auth.groovy b/regression-test/suites/auth_call/test_dml_select_udf_auth.groovy
index b2f565fc02ef8ac..6aa74784969e300 100644
--- a/regression-test/suites/auth_call/test_dml_select_udf_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_select_udf_auth.groovy
@@ -34,7 +34,7 @@ suite("test_dml_select_udf_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy b/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy
index 26ee5526f1ea69e..02b0b59ff390f65 100644
--- a/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_stream_load_auth.groovy
@@ -29,7 +29,7 @@ suite("test_dml_stream_load_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_dml_update_table_auth.groovy b/regression-test/suites/auth_call/test_dml_update_table_auth.groovy
index 153dad5026a9301..eb258df2df28006 100644
--- a/regression-test/suites/auth_call/test_dml_update_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_dml_update_table_auth.groovy
@@ -30,7 +30,7 @@ suite("test_dml_update_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy b/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy
index 3a736fe47038169..c05da39ce2e06cd 100644
--- a/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy
+++ b/regression-test/suites/auth_call/test_grant_show_view_priv_auth.groovy
@@ -32,7 +32,7 @@ suite("test_grant_show_view_priv_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_hive_base_case_auth.groovy b/regression-test/suites/auth_call/test_hive_base_case_auth.groovy
index 271015bec7c1546..0fb0d422ebd2c09 100644
--- a/regression-test/suites/auth_call/test_hive_base_case_auth.groovy
+++ b/regression-test/suites/auth_call/test_hive_base_case_auth.groovy
@@ -29,7 +29,7 @@ suite("test_hive_base_case_auth", "p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
String enabled = context.config.otherConfigs.get("enableHiveTest")
diff --git a/regression-test/suites/auth_call/test_show_backend_auth.groovy b/regression-test/suites/auth_call/test_show_backend_auth.groovy
index adccf3637e6e2a5..13e1fa4fbe480c5 100644
--- a/regression-test/suites/auth_call/test_show_backend_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_backend_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_backend_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_broker_auth.groovy b/regression-test/suites/auth_call/test_show_broker_auth.groovy
index d8fa9bd10ebe010..a364ad5b33146d8 100644
--- a/regression-test/suites/auth_call/test_show_broker_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_broker_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_broker_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_charset_auth.groovy b/regression-test/suites/auth_call/test_show_charset_auth.groovy
index 3ca23f7f6becf74..14991bc99f9e384 100644
--- a/regression-test/suites/auth_call/test_show_charset_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_charset_auth.groovy
@@ -27,7 +27,7 @@ suite("test_show_no_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_convert_light_sc_auth.groovy b/regression-test/suites/auth_call/test_show_convert_light_sc_auth.groovy
index d54862f1710845b..6f8387d892925b6 100644
--- a/regression-test/suites/auth_call/test_show_convert_light_sc_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_convert_light_sc_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_convert_light_sc_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_create_table_auth.groovy b/regression-test/suites/auth_call/test_show_create_table_auth.groovy
index 2a85ea42e6c8da1..166179bae5f6bec 100644
--- a/regression-test/suites/auth_call/test_show_create_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_create_table_auth.groovy
@@ -28,7 +28,7 @@ suite("test_show_create_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_data_auth.groovy b/regression-test/suites/auth_call/test_show_data_auth.groovy
index c6f3b6dd1536ba0..951ba564d42fcf6 100644
--- a/regression-test/suites/auth_call/test_show_data_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_data_auth.groovy
@@ -29,7 +29,7 @@ suite("test_show_data_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_database_id_auth.groovy b/regression-test/suites/auth_call/test_show_database_id_auth.groovy
index e30dc8d0db74a49..d9d131ee7793b49 100644
--- a/regression-test/suites/auth_call/test_show_database_id_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_database_id_auth.groovy
@@ -28,7 +28,7 @@ suite("test_show_database_id_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy b/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy
index 727d705664311b2..ad560fbf18f79f5 100644
--- a/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_dynamic_table_auth.groovy
@@ -28,7 +28,7 @@ suite("test_show_dynamic_table_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_frontend_auth.groovy b/regression-test/suites/auth_call/test_show_frontend_auth.groovy
index f4a9bc15b074ecc..87f8e9c7c09a686 100644
--- a/regression-test/suites/auth_call/test_show_frontend_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_frontend_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_frontend_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_grant_auth.groovy b/regression-test/suites/auth_call/test_show_grant_auth.groovy
index 1223faf6739d4a8..efd2e84e38a9f70 100644
--- a/regression-test/suites/auth_call/test_show_grant_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_grant_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_grant_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_proc_auth.groovy b/regression-test/suites/auth_call/test_show_proc_auth.groovy
index 1daf8d97f703f25..4608c64ca670160 100644
--- a/regression-test/suites/auth_call/test_show_proc_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_proc_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_proc_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_query_stats_auth.groovy b/regression-test/suites/auth_call/test_show_query_stats_auth.groovy
index ba951a47465ee3a..7552038a7432b24 100644
--- a/regression-test/suites/auth_call/test_show_query_stats_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_query_stats_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_query_stats_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_tablet_auth.groovy b/regression-test/suites/auth_call/test_show_tablet_auth.groovy
index 3ac938d84622280..eb73a7b906be113 100644
--- a/regression-test/suites/auth_call/test_show_tablet_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_tablet_auth.groovy
@@ -28,7 +28,7 @@ suite("test_show_tablet_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_call/test_show_typecast_auth.groovy b/regression-test/suites/auth_call/test_show_typecast_auth.groovy
index 8df4a2e2dc18c2d..9e5d7186f01e17d 100644
--- a/regression-test/suites/auth_call/test_show_typecast_auth.groovy
+++ b/regression-test/suites/auth_call/test_show_typecast_auth.groovy
@@ -26,7 +26,7 @@ suite("test_show_typecast_auth","p0,auth_call") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
try_sql("DROP USER ${user}")
diff --git a/regression-test/suites/auth_p0/test_backends_auth.groovy b/regression-test/suites/auth_p0/test_backends_auth.groovy
index 753ae837c776e92..db76b2740fccaf1 100644
--- a/regression-test/suites/auth_p0/test_backends_auth.groovy
+++ b/regression-test/suites/auth_p0/test_backends_auth.groovy
@@ -29,7 +29,7 @@ suite("test_backends_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_catalogs_auth.groovy b/regression-test/suites/auth_p0/test_catalogs_auth.groovy
index 96ebcef7cf81cb6..1b67282d8fe2065 100644
--- a/regression-test/suites/auth_p0/test_catalogs_auth.groovy
+++ b/regression-test/suites/auth_p0/test_catalogs_auth.groovy
@@ -36,7 +36,7 @@ suite("test_catalogs_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_frontends_auth.groovy b/regression-test/suites/auth_p0/test_frontends_auth.groovy
index 21fff527518e2b9..0ac96e5c653827d 100644
--- a/regression-test/suites/auth_p0/test_frontends_auth.groovy
+++ b/regression-test/suites/auth_p0/test_frontends_auth.groovy
@@ -29,7 +29,7 @@ suite("test_frontends_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_frontends_disks_auth.groovy b/regression-test/suites/auth_p0/test_frontends_disks_auth.groovy
index 3767fdde0a5e92b..f46ead3256a52c4 100644
--- a/regression-test/suites/auth_p0/test_frontends_disks_auth.groovy
+++ b/regression-test/suites/auth_p0/test_frontends_disks_auth.groovy
@@ -29,7 +29,7 @@ suite("test_frontends_disks_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_master_slave_consistency_auth.groovy b/regression-test/suites/auth_p0/test_master_slave_consistency_auth.groovy
index 90228ebf3a5130e..0fc6b3a40631423 100644
--- a/regression-test/suites/auth_p0/test_master_slave_consistency_auth.groovy
+++ b/regression-test/suites/auth_p0/test_master_slave_consistency_auth.groovy
@@ -110,7 +110,7 @@ suite ("test_follower_consistent_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
logger.info("url_tmp1:" + url_tmp1)
diff --git a/regression-test/suites/auth_p0/test_mtmv_auth.groovy b/regression-test/suites/auth_p0/test_mtmv_auth.groovy
index 52ecbebb70b268b..a190edaa0224daf 100644
--- a/regression-test/suites/auth_p0/test_mtmv_auth.groovy
+++ b/regression-test/suites/auth_p0/test_mtmv_auth.groovy
@@ -58,7 +58,7 @@ suite("test_mtmv_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_partition_values_tvf_auth.groovy b/regression-test/suites/auth_p0/test_partition_values_tvf_auth.groovy
index 3f0ae7ea8d524c4..b9ce9f4364619ec 100644
--- a/regression-test/suites/auth_p0/test_partition_values_tvf_auth.groovy
+++ b/regression-test/suites/auth_p0/test_partition_values_tvf_auth.groovy
@@ -44,7 +44,7 @@ suite("test_partition_values_tvf_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_partitions_auth.groovy b/regression-test/suites/auth_p0/test_partitions_auth.groovy
index 0b769f115678456..1a398b84b4e84d4 100644
--- a/regression-test/suites/auth_p0/test_partitions_auth.groovy
+++ b/regression-test/suites/auth_p0/test_partitions_auth.groovy
@@ -48,7 +48,7 @@ suite("test_partitions_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_query_tvf_auth.groovy b/regression-test/suites/auth_p0/test_query_tvf_auth.groovy
index 746eb47ce5b870e..6353ca142a9067e 100644
--- a/regression-test/suites/auth_p0/test_query_tvf_auth.groovy
+++ b/regression-test/suites/auth_p0/test_query_tvf_auth.groovy
@@ -48,7 +48,7 @@ suite("test_query_tvf_auth", "p0,auth,external,external_docker") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${dorisuser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${dorisuser}""";
}
sql """grant select_priv on regression_test to ${dorisuser}"""
diff --git a/regression-test/suites/auth_p0/test_select_column_auth.groovy b/regression-test/suites/auth_p0/test_select_column_auth.groovy
index 36cc2a0a09cf1c2..296b47975fb707e 100644
--- a/regression-test/suites/auth_p0/test_select_column_auth.groovy
+++ b/regression-test/suites/auth_p0/test_select_column_auth.groovy
@@ -37,7 +37,7 @@ suite("test_select_column_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """create database ${dbName}"""
sql("""use ${dbName}""")
@@ -69,7 +69,7 @@ suite("test_select_column_auth","p0,auth") {
(3, "333");
"""
sql """refresh MATERIALIZED VIEW ${dbName}.${mtmv_name} auto"""
- waitingMTMVTaskFinishedByMvName(mtmv_name)
+ waitingMTMVTaskFinishedByMvName(mtmv_name, dbName)
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_select_count_auth.groovy b/regression-test/suites/auth_p0/test_select_count_auth.groovy
index ccea1a4a580098a..47a199aaca22917 100644
--- a/regression-test/suites/auth_p0/test_select_count_auth.groovy
+++ b/regression-test/suites/auth_p0/test_select_count_auth.groovy
@@ -29,7 +29,7 @@ suite("test_select_count_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/auth_p0/test_select_view_auth.groovy b/regression-test/suites/auth_p0/test_select_view_auth.groovy
index 9a5020ea163ce69..6932fbbb58ded28 100644
--- a/regression-test/suites/auth_p0/test_select_view_auth.groovy
+++ b/regression-test/suites/auth_p0/test_select_view_auth.groovy
@@ -37,7 +37,7 @@ suite("test_select_view_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """create database ${dbName}"""
sql("""use ${dbName}""")
diff --git a/regression-test/suites/auth_p0/test_use_encryptkey_auth.groovy b/regression-test/suites/auth_p0/test_use_encryptkey_auth.groovy
index 965bd4b3b162bbd..258c1726ebfa62d 100644
--- a/regression-test/suites/auth_p0/test_use_encryptkey_auth.groovy
+++ b/regression-test/suites/auth_p0/test_use_encryptkey_auth.groovy
@@ -32,7 +32,7 @@ suite("test_use_encryptkey_auth","p0,auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """CREATE ENCRYPTKEY ${key} AS 'ABCD123456789'"""
diff --git a/regression-test/suites/auth_up_down_p0/load.groovy b/regression-test/suites/auth_up_down_p0/load.groovy
index 7ac11b627abe9e2..734293185897e55 100644
--- a/regression-test/suites/auth_up_down_p0/load.groovy
+++ b/regression-test/suites/auth_up_down_p0/load.groovy
@@ -50,8 +50,8 @@ suite("test_upgrade_downgrade_prepare_auth","p0,auth,restart_fe") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user1}""";
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user2}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user1}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user2}""";
}
try_sql """drop table if exists ${dbName}.${tableName1}"""
diff --git a/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy b/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy
index 8e548eb27ea87c1..eae3d1fb3591fe4 100644
--- a/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy
+++ b/regression-test/suites/correctness_p0/test_mv_case/test_mv_case.groovy
@@ -16,6 +16,7 @@
// under the License.
suite("test_mv_case") {
+
sql """drop table if exists test_table_aaa2;"""
sql """CREATE TABLE `test_table_aaa2` (
`ordernum` varchar(65533) NOT NULL ,
@@ -29,7 +30,7 @@ suite("test_mv_case") {
"replication_allocation" = "tag.location.default: 1"
);"""
sql """DROP MATERIALIZED VIEW IF EXISTS ods_zn_dnt_max1 ON test_table_aaa2;"""
- createMV("""create materialized view ods_zn_dnt_max1 as
+ create_sync_mv(context.dbName, "test_table_aaa2", "ods_zn_dnt_max1", """
select ordernum,max(dnt) as dnt from test_table_aaa2
group by ordernum
ORDER BY ordernum;""")
@@ -92,7 +93,7 @@ suite("test_mv_case") {
)
"""
sql """insert into tb1 select id,map_agg(a, b) from(select 123 id,3 a,'5' b union all select 123 id, 6 a, '8' b) aa group by id"""
- createMV ("""CREATE MATERIALIZED VIEW mv1 BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 MINUTE DUPLICATE KEY(info_id) DISTRIBUTED BY HASH(`info_id`) BUCKETS 2 PROPERTIES (
+ sql"""CREATE MATERIALIZED VIEW mv1 BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 MINUTE DUPLICATE KEY(info_id) DISTRIBUTED BY HASH(`info_id`) BUCKETS 2 PROPERTIES (
"replication_allocation" = "tag.location.default: 1",
"min_load_replica_num" = "-1",
"is_being_synced" = "false",
@@ -112,8 +113,9 @@ suite("test_mv_case") {
cast(a.id as bigint) info_id,
map_infos
from
- tb1 a;""")
- createMV ("""CREATE MATERIALIZED VIEW mv2 BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 MINUTE DUPLICATE KEY(info_id) DISTRIBUTED BY HASH(`info_id`) BUCKETS 2 PROPERTIES (
+ tb1 a;"""
+ waitingMTMVTaskFinishedByMvName("mv1")
+ sql """CREATE MATERIALIZED VIEW mv2 BUILD IMMEDIATE REFRESH COMPLETE ON SCHEDULE EVERY 10 MINUTE DUPLICATE KEY(info_id) DISTRIBUTED BY HASH(`info_id`) BUCKETS 2 PROPERTIES (
"replication_allocation" = "tag.location.default: 1",
"min_load_replica_num" = "-1",
"is_being_synced" = "false",
@@ -132,6 +134,7 @@ suite("test_mv_case") {
info_id,
map_infos
from
- mv1 a;""")
+ mv1 a;"""
+ waitingMTMVTaskFinishedByMvName("mv2")
qt_select_mv """ select * from mv2 """
}
diff --git a/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy b/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy
index 295b195aa589547..7b6069968f4a0ae 100644
--- a/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy
+++ b/regression-test/suites/ddl_p0/create_view_nereids/create_view_use_mv.groovy
@@ -48,8 +48,8 @@ suite("create_view_use_mv") {
(3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'),
(1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17');"""
- createMV("""
- CREATE MATERIALIZED VIEW t_mv_mv AS select
+ create_sync_mv(context.dbName, "orders", "t_mv_mv", """
+ select
o_orderkey,
sum(o_totalprice) as sum_total,
max(o_totalprice) as max_total,
diff --git a/regression-test/suites/external_table_p0/jdbc/test_jdbc_call.groovy b/regression-test/suites/external_table_p0/jdbc/test_jdbc_call.groovy
index e9e00b7084fba03..7f4d7403c3d43a5 100644
--- a/regression-test/suites/external_table_p0/jdbc/test_jdbc_call.groovy
+++ b/regression-test/suites/external_table_p0/jdbc/test_jdbc_call.groovy
@@ -118,8 +118,8 @@ suite("test_jdbc_call", "p0,external,doris,external_docker,external_docker_doris
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user1}""";
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user2}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user1}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user2}""";
}
def result1 = connect("${user1}", "", context.config.jdbcUrl) {
diff --git a/regression-test/suites/external_table_p0/lower_case/test_conflict_name.groovy b/regression-test/suites/external_table_p0/lower_case/test_conflict_name.groovy
index c8ed6c4cbbc8881..92fe356dfd6c350 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_conflict_name.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_conflict_name.groovy
@@ -32,7 +32,7 @@ suite("test_conflict_name", "p0,external,doris,meta_names_mapping,external_docke
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_include.groovy b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_include.groovy
index 63f18e358c71bf6..91a56f7f317a693 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_include.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_include.groovy
@@ -32,7 +32,7 @@ suite("test_lower_case_meta_include", "p0,external,doris,external_docker,externa
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_show_and_select.groovy b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_show_and_select.groovy
index 72e945ea8ffd59a..8853d169a13bae3 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_show_and_select.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_show_and_select.groovy
@@ -48,7 +48,7 @@ suite("test_lower_case_meta_show_and_select", "p0,external,doris,external_docker
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_grant.groovy b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_grant.groovy
index ab47a1e23bd63a2..78baa9aa4384526 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_grant.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_grant.groovy
@@ -86,7 +86,7 @@ suite("test_lower_case_meta_with_lower_table_conf_auth", "p0,external,doris,exte
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
@@ -134,7 +134,7 @@ suite("test_lower_case_meta_with_lower_table_conf_auth", "p0,external,doris,exte
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_show_and_select.groovy b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_show_and_select.groovy
index 42c6fd08203e4f6..13750535628c440 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_show_and_select.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_lower_case_meta_with_lower_table_conf_show_and_select.groovy
@@ -48,7 +48,7 @@ suite("test_lower_case_meta_with_lower_table_conf_show_and_select", "p0,external
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_lower_case_mtmv.groovy b/regression-test/suites/external_table_p0/lower_case/test_lower_case_mtmv.groovy
index 4cc8593e459a6c1..67081ed3afe12c5 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_lower_case_mtmv.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_lower_case_mtmv.groovy
@@ -32,7 +32,7 @@ suite("test_lower_case_mtmv", "p0,external,doris,external_docker,external_docker
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_meta_cache_select_without_refresh.groovy b/regression-test/suites/external_table_p0/lower_case/test_meta_cache_select_without_refresh.groovy
index b677133015c996e..c2073f9864cef6d 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_meta_cache_select_without_refresh.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_meta_cache_select_without_refresh.groovy
@@ -32,7 +32,7 @@ suite("test_meta_cache_select_without_refresh", "p0,external,doris,external_dock
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_meta_names_mapping.groovy b/regression-test/suites/external_table_p0/lower_case/test_meta_names_mapping.groovy
index 1ae22e1ba99020a..a033b7e59f8691d 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_meta_names_mapping.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_meta_names_mapping.groovy
@@ -32,7 +32,7 @@ suite("test_meta_names_mapping", "p0,external,doris,meta_names_mapping,external_
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/test_timing_refresh_catalog.groovy b/regression-test/suites/external_table_p0/lower_case/test_timing_refresh_catalog.groovy
index 5e0386330e3a515..a353e20d685074e 100644
--- a/regression-test/suites/external_table_p0/lower_case/test_timing_refresh_catalog.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/test_timing_refresh_catalog.groovy
@@ -53,7 +53,7 @@ suite("test_timing_refresh_catalog", "p0,external,doris,external_docker,external
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/lower_case/upgrade/load.groovy b/regression-test/suites/external_table_p0/lower_case/upgrade/load.groovy
index 053fad17785a2f2..8be76c3e0f44a4e 100644
--- a/regression-test/suites/external_table_p0/lower_case/upgrade/load.groovy
+++ b/regression-test/suites/external_table_p0/lower_case/upgrade/load.groovy
@@ -48,7 +48,7 @@ suite("test_upgrade_lower_case_catalog_prepare", "p0,external,doris,external_doc
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${jdbcUser}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${jdbcUser}""";
}
sql """grant all on *.*.* to ${jdbcUser}"""
diff --git a/regression-test/suites/external_table_p0/tvf/test_insert_from_tvf_with_common_user.groovy b/regression-test/suites/external_table_p0/tvf/test_insert_from_tvf_with_common_user.groovy
index 103158c224c13f2..f0ae51430fd4dd0 100644
--- a/regression-test/suites/external_table_p0/tvf/test_insert_from_tvf_with_common_user.groovy
+++ b/regression-test/suites/external_table_p0/tvf/test_insert_from_tvf_with_common_user.groovy
@@ -93,7 +93,7 @@ suite("test_insert_from_tvf_with_common_user", "p0,external,external_docker") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${common_user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${common_user}""";
}
connect("${common_user}", '12345', context.config.jdbcUrl) {
diff --git a/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy b/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy
index 92091c18926bf9a..1474ec0890b5fd4 100644
--- a/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy
+++ b/regression-test/suites/external_table_p0/tvf/test_s3_tvf_with_resource.groovy
@@ -203,7 +203,7 @@ suite("test_s3_tvf_with_resource", "p0,external,external_docker") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
// not have usage priv, can not select tvf with resource
connect(user, "${pwd}", url) {
diff --git a/regression-test/suites/external_table_p2/tvf/test_iceberg_meta.groovy b/regression-test/suites/external_table_p2/tvf/test_iceberg_meta.groovy
index 557eaf5b061d70f..3fc898b865fb170 100644
--- a/regression-test/suites/external_table_p2/tvf/test_iceberg_meta.groovy
+++ b/regression-test/suites/external_table_p2/tvf/test_iceberg_meta.groovy
@@ -63,7 +63,7 @@ suite("test_iceberg_meta", "p2,external,iceberg,external_remote,external_remote_
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """grant select_priv on regression_test to ${user}"""
diff --git a/regression-test/suites/javaudf_p0/test_javaudf_auth.groovy b/regression-test/suites/javaudf_p0/test_javaudf_auth.groovy
index 0729f14bb333f15..7f02c9218550ab0 100644
--- a/regression-test/suites/javaudf_p0/test_javaudf_auth.groovy
+++ b/regression-test/suites/javaudf_p0/test_javaudf_auth.groovy
@@ -49,7 +49,7 @@ suite("test_javaudf_auth") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql """USE ${dbName}"""
diff --git a/regression-test/suites/load_p0/routine_load/test_routine_load_with_user.groovy b/regression-test/suites/load_p0/routine_load/test_routine_load_with_user.groovy
index 73cce57822fab3f..73641f46d018c24 100644
--- a/regression-test/suites/load_p0/routine_load/test_routine_load_with_user.groovy
+++ b/regression-test/suites/load_p0/routine_load/test_routine_load_with_user.groovy
@@ -65,7 +65,7 @@ suite("test_routine_load_with_user","p0") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
try {
def storageVaults = (sql " SHOW STORAGE VAULT; ").stream().map(row -> row[0]).collect(Collectors.toSet())
diff --git a/regression-test/suites/load_p0/stream_load/test_stream_load.groovy b/regression-test/suites/load_p0/stream_load/test_stream_load.groovy
index 54731a949584bed..1bbf6033ef8ca05 100644
--- a/regression-test/suites/load_p0/stream_load/test_stream_load.groovy
+++ b/regression-test/suites/load_p0/stream_load/test_stream_load.groovy
@@ -1059,7 +1059,7 @@ suite("test_stream_load", "p0") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO common_user""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO common_user""";
}
streamLoad {
diff --git a/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy b/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy
index bbd532a76259cce..1df859d03d144a5 100644
--- a/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy
+++ b/regression-test/suites/load_p0/stream_load/test_stream_load_move_memtable.groovy
@@ -937,7 +937,7 @@ suite("test_stream_load_move_memtable", "p0") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ddd""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ddd""";
}
streamLoad {
diff --git a/regression-test/suites/manager/test_manager_interface_3.groovy b/regression-test/suites/manager/test_manager_interface_3.groovy
index 22af98d5648ca77..b7baa8a869d94a0 100644
--- a/regression-test/suites/manager/test_manager_interface_3.groovy
+++ b/regression-test/suites/manager/test_manager_interface_3.groovy
@@ -89,8 +89,8 @@ suite('test_manager_interface_3',"p0") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user1}""";
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user2}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user1}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user2}""";
}
connect(user1, "${pwd}", url) {
@@ -412,7 +412,7 @@ suite('test_manager_interface_3',"p0") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
List> result = sql """ show resources """
@@ -609,7 +609,7 @@ suite('test_manager_interface_3',"p0") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
connect(user, "${pwd}", url) {
diff --git a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy
index 36c0d3f120e1090..8dd16ea571f1ec8 100644
--- a/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy
+++ b/regression-test/suites/mtmv_p0/test_iceberg_mtmv.groovy
@@ -107,35 +107,35 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-26 01:02:03', 1), ('2024-10-27 01:02:03', 2), ('2024-10-27 21:02:03', 3)"""
sql """CREATE MATERIALIZED VIEW ${mvName1} BUILD DEFERRED REFRESH AUTO ON MANUAL partition by(`ts`) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') as SELECT * FROM ${catalog_name}.${icebergDb}.${icebergTable1}"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} complete"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh1 "select * from ${mvName1} order by value"
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-26 21:02:03', 4)"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} auto"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh2 """select * from ${mvName1} order by value"""
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-26 01:22:03', 5), ('2024-10-27 01:12:03', 6);"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} partitions(p_20241026000000_20241027000000);"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh3 """select * from ${mvName1} order by value"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} auto"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh4 """select * from ${mvName1} order by value"""
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values ('2024-10-28 01:22:03', 7);"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} partitions(p_20241026000000_20241027000000);"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh5 """select * from ${mvName1} order by value"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} auto"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh6 """select * from ${mvName1} order by value"""
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable1} values (null, 8);"""
sql """REFRESH MATERIALIZED VIEW ${mvName1} auto"""
- waitingMTMVTaskFinishedByMvName(mvName1)
+ waitingMTMVTaskFinishedByMvName(mvName1, dbName)
qt_test_ts_refresh_null """select * from ${mvName1} order by value"""
def showPartitionsResult = sql """show partitions from ${mvName1}"""
@@ -176,25 +176,25 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-08-26', 1), ('2024-09-17', 2), ('2024-09-27', 3);"""
sql """CREATE MATERIALIZED VIEW ${mvName2} BUILD DEFERRED REFRESH AUTO ON MANUAL partition by(`d`) DISTRIBUTED BY RANDOM BUCKETS 2 PROPERTIES ('replication_num' = '1') as SELECT * FROM ${catalog_name}.${icebergDb}.${icebergTable2}"""
sql """REFRESH MATERIALIZED VIEW ${mvName2} complete"""
- waitingMTMVTaskFinishedByMvName(mvName2)
+ waitingMTMVTaskFinishedByMvName(mvName2, dbName)
qt_test_d_refresh1 "select * from ${mvName2} order by value"
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-09-01', 4);"""
sql """REFRESH MATERIALIZED VIEW ${mvName2} auto"""
- waitingMTMVTaskFinishedByMvName(mvName2)
+ waitingMTMVTaskFinishedByMvName(mvName2, dbName)
qt_test_d_refresh2 "select * from ${mvName2} order by value"
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-08-22', 5), ('2024-09-30', 6);"""
sql """REFRESH MATERIALIZED VIEW ${mvName2} partitions(p_20240801_20240901);"""
- waitingMTMVTaskFinishedByMvName(mvName2)
+ waitingMTMVTaskFinishedByMvName(mvName2, dbName)
qt_test_d_refresh3 "select * from ${mvName2} order by value"
sql """REFRESH MATERIALIZED VIEW ${mvName2} partitions(p_20240901_20241001);"""
- waitingMTMVTaskFinishedByMvName(mvName2)
+ waitingMTMVTaskFinishedByMvName(mvName2, dbName)
qt_test_d_refresh4 "select * from ${mvName2} order by value"
sql """insert into ${catalog_name}.${icebergDb}.${icebergTable2} values ('2024-10-28', 7);"""
sql """REFRESH MATERIALIZED VIEW ${mvName2} auto"""
- waitingMTMVTaskFinishedByMvName(mvName2)
+ waitingMTMVTaskFinishedByMvName(mvName2, dbName)
qt_test_d_refresh5 "select * from ${mvName2} order by value"
showPartitionsResult = sql """show partitions from ${mvName2}"""
@@ -240,7 +240,7 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_
// refresh one partiton
sql """REFRESH MATERIALIZED VIEW ${mvName} partitions(p_20240101000000_20240102000000);"""
- waitingMTMVTaskFinishedByMvName(mvName)
+ waitingMTMVTaskFinishedByMvName(mvName, dbName)
order_qt_refresh_one_partition "SELECT * FROM ${mvName} "
def explainOnePartition = sql """ explain ${mvSql} """
logger.info("explainOnePartition: " + explainOnePartition.toString())
@@ -250,7 +250,7 @@ suite("test_iceberg_mtmv", "p0,external,iceberg,external_docker,external_docker_
//refresh auto
sql """REFRESH MATERIALIZED VIEW ${mvName} auto"""
- waitingMTMVTaskFinishedByMvName(mvName)
+ waitingMTMVTaskFinishedByMvName(mvName, dbName)
order_qt_refresh_auto "SELECT * FROM ${mvName} "
def explainAllPartition = sql """ explain ${mvSql}; """
logger.info("explainAllPartition: " + explainAllPartition.toString())
diff --git a/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy b/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy
index a3ac1c048d30dab..7a77cdc4590a4c6 100644
--- a/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy
+++ b/regression-test/suites/mtmv_p0/test_paimon_olap_rewrite_mtmv.groovy
@@ -87,6 +87,10 @@ suite("test_paimon_olap_rewrite_mtmv", "p0,external,mtmv,external_docker,externa
def explainOnePartition = sql """ explain ${mvSql} """
logger.info("explainOnePartition: " + explainOnePartition.toString())
+
+ def explain_memo_plan = sql """ explain memo plan ${mvSql} """
+ logger.info("explain_memo_plan: " + explain_memo_plan.toString())
+
assertTrue(explainOnePartition.toString().contains("VUNION"))
order_qt_refresh_one_partition_rewrite "${mvSql}"
@@ -104,6 +108,10 @@ suite("test_paimon_olap_rewrite_mtmv", "p0,external,mtmv,external_docker,externa
def explainAllPartition = sql """ explain ${mvSql}; """
logger.info("explainAllPartition: " + explainAllPartition.toString())
+
+ def explainMemoPlan = sql """ explain memo plan ${mvSql}; """
+ logger.info("explainMemoPlan: " + explainMemoPlan.toString())
+
assertTrue(explainAllPartition.toString().contains("VOlapScanNode"))
order_qt_refresh_all_partition_rewrite "${mvSql}"
diff --git a/regression-test/suites/mv_p0/no_await/no_await.groovy b/regression-test/suites/mv_p0/no_await/no_await.groovy
index 866e4fdd5d35469..3eab03aa7e4adc9 100644
--- a/regression-test/suites/mv_p0/no_await/no_await.groovy
+++ b/regression-test/suites/mv_p0/no_await/no_await.groovy
@@ -19,6 +19,8 @@ import org.codehaus.groovy.runtime.IOGroovyMethods
suite ("no_await") {
+ String db = context.config.getDbNameByFile(context.file)
+
def tblName = "agg_have_dup_base_no_await"
def waitDrop = {
def try_times = 1000
@@ -58,92 +60,93 @@ suite ("no_await") {
"""
sql "insert into ${tblName} select e1, -4, -4, 'd' from (select 1 k1) as t lateral view explode_numbers(10000) tmp1 as e1;"
// do not await
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
+
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
waitDrop()
- sql "create materialized view k12s3m as select k1,sum(k2),max(k2) from ${tblName} group by k1;"
+ create_sync_mv(db, tblName, "k12s3m", """select k1,sum(k2),max(k2) from ${tblName} group by k1;""")
sql "insert into ${tblName} select -4, -4, -4, \'d\'"
qt_mv "select sum(k1) from ${tblName}"
}
diff --git a/regression-test/suites/mv_p0/unique/unique_rewrite.groovy b/regression-test/suites/mv_p0/unique_rewrite/unique_rewrite.groovy
similarity index 100%
rename from regression-test/suites/mv_p0/unique/unique_rewrite.groovy
rename to regression-test/suites/mv_p0/unique_rewrite/unique_rewrite.groovy
diff --git a/regression-test/suites/nereids_p0/authorization/column_authorization.groovy b/regression-test/suites/nereids_p0/authorization/column_authorization.groovy
index c2e22f10c22678c..eea353368c96841 100644
--- a/regression-test/suites/nereids_p0/authorization/column_authorization.groovy
+++ b/regression-test/suites/nereids_p0/authorization/column_authorization.groovy
@@ -49,7 +49,7 @@ suite("column_authorization") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user1}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user1}""";
}
sql 'sync'
diff --git a/regression-test/suites/nereids_p0/authorization/view_authorization.groovy b/regression-test/suites/nereids_p0/authorization/view_authorization.groovy
index 51503c3cd2d7377..fa7f56c1e09f0b0 100644
--- a/regression-test/suites/nereids_p0/authorization/view_authorization.groovy
+++ b/regression-test/suites/nereids_p0/authorization/view_authorization.groovy
@@ -52,7 +52,7 @@ suite("view_authorization") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user1}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user1}""";
}
sql 'sync'
diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy
index e7fb5f3da6c435e..765d12084266070 100644
--- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy
+++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy
@@ -297,7 +297,7 @@ suite("parse_sql_from_sql_cache") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_cache_user1"""
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_cache_user1"""
}
createTestTable "test_use_plan_cache12"
@@ -340,7 +340,7 @@ suite("parse_sql_from_sql_cache") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_cache_user2"""
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_cache_user2"""
}
createTestTable "test_use_plan_cache13"
@@ -397,7 +397,7 @@ suite("parse_sql_from_sql_cache") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_cache_user3"""
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_cache_user3"""
}
createTestTable "test_use_plan_cache14"
@@ -460,7 +460,7 @@ suite("parse_sql_from_sql_cache") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_cache_user4"""
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_cache_user4"""
}
sql "sync"
diff --git a/regression-test/suites/nereids_p0/insert_into_table/insert_auth.groovy b/regression-test/suites/nereids_p0/insert_into_table/insert_auth.groovy
index 4c9968746d13c1a..d146885a7bf936c 100644
--- a/regression-test/suites/nereids_p0/insert_into_table/insert_auth.groovy
+++ b/regression-test/suites/nereids_p0/insert_into_table/insert_auth.groovy
@@ -53,7 +53,7 @@ suite('nereids_insert_auth') {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
connect(user, "${pwd}", url) {
diff --git a/regression-test/suites/nereids_rules_p0/mv/date_trunc/mv_with_date_trunc.groovy b/regression-test/suites/nereids_rules_p0/mv/date_trunc/mv_with_date_trunc.groovy
index 58b2e7bbdd8ca34..e88f4132eacfba0 100644
--- a/regression-test/suites/nereids_rules_p0/mv/date_trunc/mv_with_date_trunc.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/date_trunc/mv_with_date_trunc.groovy
@@ -1447,7 +1447,7 @@ suite("mv_with_date_trunc") {
logger.info("lineitem table stats: " + result)
result = sql """show index stats lineitem lineitem"""
logger.info("lineitem index stats: " + result)
- mv_rewrite_success(query4_0, "mv4_0")
+ mv_rewrite_success(query4_0, "mv4_0", true, is_partition_statistics_ready(db, ["lineitem", "mv4_0"]))
order_qt_query4_0_after "${query4_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0"""
diff --git a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy
index 4870ec99e659c06..1972c2d505bfab5 100644
--- a/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy
+++ b/regression-test/suites/nereids_rules_p0/mv/nested_mtmv/nested_mtmv.groovy
@@ -742,10 +742,12 @@ suite("nested_mtmv") {
mv_rewrite_any_success(sql_2, [mv_1, mv_2])
compare_res(sql_2 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13")
- mv_rewrite_any_success(sql_3, [mv_3, mv_4])
+ // level 1 maybe use mv_1 and mv_2, this also meets expectation
+ mv_rewrite_any_success(sql_3, [mv_3, mv_4, mv_1, mv_2])
compare_res(sql_3 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13")
- mv_rewrite_any_success(sql_4, [mv_3, mv_4])
+ // level 1 maybe use mv_1 and mv_2, this also meets expectation
+ mv_rewrite_any_success(sql_4, [mv_3, mv_4, mv_1, mv_2])
compare_res(sql_4 + " order by 1,2,3,4,5,6,7,8,9,10,11,12,13")
mv_rewrite_any_success(sql_5, [mv_3, mv_4, mv_5])
diff --git a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy
index 7a49dbc4b37118c..32e1684fdc90733 100644
--- a/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy
+++ b/regression-test/suites/nereids_syntax_p0/mv/newMv/multi_slot4.groovy
@@ -33,21 +33,29 @@ suite ("multi_slot4") {
"""
sql "insert into multi_slot4 select 1,1,1,'a';"
+ sql "insert into multi_slot4 select 1,1,1,'a';"
+ sql "insert into multi_slot4 select 1,1,1,'a';"
+ sql "insert into multi_slot4 select 2,2,2,'b';"
+ sql "insert into multi_slot4 select 2,2,2,'b';"
sql "insert into multi_slot4 select 2,2,2,'b';"
sql "insert into multi_slot4 select 3,-3,null,'c';"
sql "insert into multi_slot4 select 3,-3,null,'c';"
+ sql "insert into multi_slot4 select 3,-3,null,'c';"
+ sql "insert into multi_slot4 select 3,-3,null,'c';"
createMV ("create materialized view k1p2ap3ps as select k1+1,sum(abs(k2+2)+k3+3) from multi_slot4 group by k1+1;")
sleep(3000)
+ sql "insert into multi_slot4 select -4,-4,-4,'d';"
+ sql "insert into multi_slot4 select -4,-4,-4,'d';"
sql "insert into multi_slot4 select -4,-4,-4,'d';"
sql "SET experimental_enable_nereids_planner=true"
sql "SET enable_fallback_to_original_planner=false"
sql "analyze table multi_slot4 with sync;"
- sql """alter table multi_slot4 modify column k1 set stats ('row_count'='5');"""
+ sql """alter table multi_slot4 modify column k1 set stats ('row_count'='13');"""
sql """set enable_stats=false;"""
diff --git a/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy b/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy
index 9550a7fec3dbd24..56f5d655255049e 100644
--- a/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy
+++ b/regression-test/suites/query_p0/show/test_show_create_materialized_view.groovy
@@ -30,12 +30,11 @@ suite("test_show_create_materialized_view", "query,arrow_flight_sql") {
DISTRIBUTED BY HASH(id) BUCKETS 5
PROPERTIES ("replication_num" = "1");
"""
-
- createMV("""CREATE MATERIALIZED VIEW ${mvName} AS
+
+ create_sync_mv(context.dbName, tableName, mvName, """
SELECT id, name, SUM(value) AS total_value
FROM ${tableName}
- GROUP BY id, name;
- """)
+ GROUP BY id, name""")
checkNereidsExecute("""SHOW CREATE MATERIALIZED VIEW ${mvName} ON ${tableName};""")
qt_cmd("""SHOW CREATE MATERIALIZED VIEW ${mvName} ON ${tableName};""")
diff --git a/regression-test/suites/query_p0/system/test_partitions_schema.groovy b/regression-test/suites/query_p0/system/test_partitions_schema.groovy
index 98b12c3705c7770..84bbeafa7fe3496 100644
--- a/regression-test/suites/query_p0/system/test_partitions_schema.groovy
+++ b/regression-test/suites/query_p0/system/test_partitions_schema.groovy
@@ -176,7 +176,7 @@ suite("test_partitions_schema") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql "GRANT SELECT_PRIV ON information_schema.partitions TO ${user}"
diff --git a/regression-test/suites/query_p0/system/test_table_options.groovy b/regression-test/suites/query_p0/system/test_table_options.groovy
index fef118a82b280ba..fd9cffcedfcad88 100644
--- a/regression-test/suites/query_p0/system/test_table_options.groovy
+++ b/regression-test/suites/query_p0/system/test_table_options.groovy
@@ -191,7 +191,7 @@ suite("test_table_options") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql "GRANT SELECT_PRIV ON information_schema.table_properties TO ${user}"
diff --git a/regression-test/suites/query_p0/system/test_table_properties.groovy b/regression-test/suites/query_p0/system/test_table_properties.groovy
index 1861ae4d6280d63..7465497ae76d080 100644
--- a/regression-test/suites/query_p0/system/test_table_properties.groovy
+++ b/regression-test/suites/query_p0/system/test_table_properties.groovy
@@ -98,7 +98,7 @@ suite("test_table_properties") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
sql "GRANT SELECT_PRIV ON information_schema.table_properties TO ${user}"
diff --git a/regression-test/suites/query_p0/test_row_policy.groovy b/regression-test/suites/query_p0/test_row_policy.groovy
index 4af498e55e3b462..c416537df15eac8 100644
--- a/regression-test/suites/query_p0/test_row_policy.groovy
+++ b/regression-test/suites/query_p0/test_row_policy.groovy
@@ -34,7 +34,7 @@ suite("test_row_policy") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO ${user}""";
}
connect(user, '123456', url) {
diff --git a/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy
index 713c470436e7fb0..a6ad20ec6230482 100644
--- a/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_dup_mv_schema_change.groovy
@@ -70,7 +70,7 @@ suite ("test_dup_mv_schema_change") {
"""
//add materialized view
- createMV("create materialized view mv1 as select date, user_id, city, age from ${tableName};")
+ create_sync_mv(context.dbName, tableName, "mv1", """select date, user_id, city, age from ${tableName}""")
// alter and test light schema change
if (!isCloudMode()) {
@@ -78,7 +78,7 @@ suite ("test_dup_mv_schema_change") {
}
//add materialized view
- createMV("create materialized view mv2 as select date, user_id, city, age, cost from ${tableName};")
+ create_sync_mv(context.dbName, tableName, "mv2", """select date, user_id, city, age, cost from ${tableName}""")
sql """ INSERT INTO ${tableName} VALUES
(2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
diff --git a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
index eba6036c30a6280..1d8fdd4d1e71929 100644
--- a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
@@ -21,20 +21,7 @@ import org.awaitility.Awaitility
suite ("test_uniq_mv_schema_change") {
def tableName = "schema_change_uniq_mv_regression_test"
- def getMVJobState = { tbName ->
- def jobStateResult = sql """ SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tbName}' ORDER BY CreateTime DESC LIMIT 1 """
- return jobStateResult[0][8]
- }
- def waitForJob = (tbName, timeout) -> {
- Awaitility.await().atMost(timeout, TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).await().until(() -> {
- String result = getMVJobState(tbName)
- if (result == "FINISHED") {
- return true;
- }
- return false;
- });
- // when timeout awaitlity will raise a exception.
- }
+
try {
String backend_id;
@@ -78,8 +65,7 @@ suite ("test_uniq_mv_schema_change") {
//add materialized view
def mvName = "mv1"
- sql "create materialized view ${mvName} as select user_id, date, city, age, sex from ${tableName};"
- waitForJob(tableName, 3000)
+ create_sync_mv(context.dbName, tableName, mvName, """select user_id, date, city, age, sex from ${tableName}""")
// alter and test light schema change
if (!isCloudMode()) {
@@ -88,8 +74,7 @@ suite ("test_uniq_mv_schema_change") {
//add materialized view
def mvName2 = "mv2"
- sql "create materialized view ${mvName2} as select user_id, date, city, age, sex, cost from ${tableName};"
- waitForJob(tableName, 3000)
+ create_sync_mv(context.dbName, tableName, mvName2, """select user_id, date, city, age, sex, cost from ${tableName};""")
sql """ INSERT INTO ${tableName} VALUES
(2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
diff --git a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy
index 56a0d7ddbf1f619..ae836fb0c652970 100644
--- a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy
+++ b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy
@@ -348,7 +348,7 @@ suite("test_crud_wlg") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_wlg_user""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_wlg_user""";
}
connect('test_wlg_user', '12345', context.config.jdbcUrl) {
@@ -739,7 +739,7 @@ suite("test_crud_wlg") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_wg_priv_user2""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_wg_priv_user2""";
}
connect('test_wg_priv_user2', '', context.config.jdbcUrl) {
qt_select_wgp_11 "select GRANTEE,WORKLOAD_GROUP_NAME,PRIVILEGE_TYPE,IS_GRANTABLE from information_schema.workload_group_privileges where grantee like '%test_wg_priv%' order by GRANTEE,WORKLOAD_GROUP_NAME,PRIVILEGE_TYPE,IS_GRANTABLE; "
diff --git a/regression-test/suites/workload_manager_p0/test_resource_tag.groovy b/regression-test/suites/workload_manager_p0/test_resource_tag.groovy
index fa7ba680143248a..d643155c0eaeb23 100644
--- a/regression-test/suites/workload_manager_p0/test_resource_tag.groovy
+++ b/regression-test/suites/workload_manager_p0/test_resource_tag.groovy
@@ -25,7 +25,7 @@ suite("test_resource_tag") {
def clusters = sql " SHOW CLUSTERS; "
assertTrue(!clusters.isEmpty())
def validCluster = clusters[0][0]
- sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO test_rg""";
+ sql """GRANT USAGE_PRIV ON CLUSTER `${validCluster}` TO test_rg""";
}
// test query