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 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 project = (PhysicalProject) 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) 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 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