This is an automated email from the ASF dual-hosted git repository. starocean999 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push: new abf51d442c8 [Enhancement](nereids)support show restore (#48784) abf51d442c8 is described below commit abf51d442c8f99bf40452bbf62dc5f358a8a53d7 Author: lsy3993 <110876560+lsy3...@users.noreply.github.com> AuthorDate: Wed Mar 19 09:45:15 2025 +0800 [Enhancement](nereids)support show restore (#48784) --- .../antlr4/org/apache/doris/nereids/DorisParser.g4 | 2 +- .../doris/nereids/parser/LogicalPlanBuilder.java | 15 ++ .../apache/doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/commands/ShowRestoreCommand.java | 214 +++++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../plans/commands/ShowRestoreCommandTest.java | 115 +++++++++++ .../cloud_p0/conf/regression-conf-custom.groovy | 1 + .../show/test_nereids_show_restore.groovy | 96 +++++++++ 8 files changed, 448 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index e67fcad0527..7abf3e8b4ad 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -296,6 +296,7 @@ supportedShowStatement (LIKE STRING_LITERAL)? #showEncryptKeys | SHOW BRIEF? CREATE TABLE name=multipartIdentifier #showCreateTable | SHOW FULL? PROCESSLIST #showProcessList + | SHOW BRIEF? RESTORE ((FROM | IN) database=identifier)? wildWhere? #showRestore | SHOW ROLES #showRoles | SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId | SHOW PRIVILEGES #showPrivileges @@ -402,7 +403,6 @@ unsupportedShowStatement wildWhere? sortClause? limitClause? #showPartitions | SHOW TABLETS FROM tableName=multipartIdentifier partitionSpec? wildWhere? sortClause? limitClause? #showTabletsFromTable - | SHOW BRIEF? RESTORE ((FROM | IN) database=multipartIdentifier)? wildWhere? #showRestore | SHOW RESOURCES wildWhere? sortClause? limitClause? #showResources | SHOW WORKLOAD GROUPS wildWhere? #showWorkloadGroups | SHOW SNAPSHOT ON repo=identifier wildWhere? #showSnapshot diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 11bbc473db8..62574ee0f68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -337,6 +337,7 @@ import org.apache.doris.nereids.DorisParser.ShowQueryProfileContext; import org.apache.doris.nereids.DorisParser.ShowQueuedAnalyzeJobsContext; import org.apache.doris.nereids.DorisParser.ShowReplicaDistributionContext; import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext; +import org.apache.doris.nereids.DorisParser.ShowRestoreContext; import org.apache.doris.nereids.DorisParser.ShowRolesContext; import org.apache.doris.nereids.DorisParser.ShowSmallFilesContext; import org.apache.doris.nereids.DorisParser.ShowSqlBlockRuleContext; @@ -648,6 +649,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowQueryProfileCommand; import org.apache.doris.nereids.trees.plans.commands.ShowQueuedAnalyzeJobsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowRestoreCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; @@ -5061,6 +5063,19 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { return new ShowRepositoriesCommand(); } + @Override + public LogicalPlan visitShowRestore(ShowRestoreContext ctx) { + String dbName = null; + Expression wildWhere = null; + if (ctx.database != null) { + dbName = ctx.database.getText(); + } + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + return new ShowRestoreCommand(dbName, wildWhere, ctx.BRIEF() != null); + } + @Override public LogicalPlan visitShowRoles(ShowRolesContext ctx) { return new ShowRolesCommand(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 087e7106ffb..f3c670e9017 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -257,6 +257,7 @@ public enum PlanType { SHOW_REPLICA_DISTRIBUTION_COMMAND, SHOW_REPLICA_STATUS_COMMAND, SHOW_REPOSITORIES_COMMAND, + SHOW_RESTORE_COMMAND, SHOW_ROLE_COMMAND, SHOW_SMALL_FILES_COMMAND, SHOW_STAGES_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowRestoreCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowRestoreCommand.java new file mode 100644 index 00000000000..590a416d9bb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowRestoreCommand.java @@ -0,0 +1,214 @@ +// 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.trees.plans.commands; + +import org.apache.doris.backup.AbstractJob; +import org.apache.doris.backup.RestoreJob; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.CaseSensibility; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.PatternMatcher; +import org.apache.doris.common.PatternMatcherWrapper; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Like; +import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * show restore command + */ +public class ShowRestoreCommand extends ShowCommand { + public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>() + .add("JobId").add("Label").add("Timestamp").add("DbName").add("State") + .add("AllowLoad").add("ReplicationNum").add("ReplicaAllocation").add("ReserveReplica") + .add("ReserveDynamicPartitionEnable").add("RestoreObjs").add("CreateTime").add("MetaPreparedTime") + .add("SnapshotFinishedTime").add("DownloadFinishedTime").add("FinishedTime").add("UnfinishedTasks") + .add("Progress").add("TaskErrMsg").add("Status").add("Timeout") + .build(); + public static final ImmutableList<String> BRIEF_TITLE_NAMES = new ImmutableList.Builder<String>() + .add("JobId").add("Label").add("Timestamp").add("DbName").add("State") + .add("AllowLoad").add("ReplicationNum").add("ReplicaAllocation").add("ReserveReplica") + .add("ReserveDynamicPartitionEnable").add("CreateTime").add("MetaPreparedTime") + .add("SnapshotFinishedTime").add("DownloadFinishedTime").add("FinishedTime").add("UnfinishedTasks") + .add("Status").add("Timeout") + .build(); + + private String dbName; + private Expression where; + private String labelValue; + private boolean isAccurateMatch; + private boolean needBriefResult; + + /** + * constructor + */ + public ShowRestoreCommand(String dbName, Expression where, boolean needBriefResult) { + super(PlanType.SHOW_RESTORE_COMMAND); + this.dbName = dbName; + this.where = where; + this.needBriefResult = needBriefResult; + } + + public ShowRestoreCommand(String dbName, Expression where) { + super(PlanType.SHOW_RESTORE_COMMAND); + this.dbName = dbName; + this.where = where; + } + + /** + * get meta for show restore + */ + private ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + List<String> titleNames = needBriefResult ? BRIEF_TITLE_NAMES : TITLE_NAMES; + for (String title : titleNames) { + builder.addColumn(new Column(title, ScalarType.STRING)); + } + return builder.build(); + } + + /** + * get label predicate for show restore + */ + @VisibleForTesting + protected Predicate<String> getLabelPredicate() throws AnalysisException { + if (null == where) { + return label -> true; + } + if (isAccurateMatch) { + return CaseSensibility.LABEL.getCaseSensibility() + ? label -> label.equals(labelValue) : label -> label.equalsIgnoreCase(labelValue); + } else { + PatternMatcher patternMatcher = PatternMatcherWrapper.createMysqlPattern( + labelValue, CaseSensibility.LABEL.getCaseSensibility()); + return patternMatcher::match; + } + } + + /** + * validate + */ + @VisibleForTesting + protected boolean validate(ConnectContext ctx) throws UserException { + if (Strings.isNullOrEmpty(dbName)) { + dbName = ctx.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + throw new AnalysisException("No database selected"); + } + } + + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, + ConnectContext.get().getQualifiedUser(), dbName); + } + + // SQL may be like : show restore from your_db_name; there is no where clause. + if (where == null) { + return true; + } + + if (!(where instanceof Like) && !(where instanceof EqualTo)) { + return false; + } + + if (where instanceof EqualTo) { + isAccurateMatch = true; + } + + // left child + if (!(where.child(0) instanceof UnboundSlot)) { + return false; + } + String leftKey = ((UnboundSlot) where.child(0)).getName(); + if (!"label".equalsIgnoreCase(leftKey)) { + return false; + } + + // right child + if (!(where.child(1) instanceof StringLikeLiteral)) { + return false; + } + labelValue = ((StringLikeLiteral) where.child(1)).getStringValue(); + if (Strings.isNullOrEmpty(labelValue)) { + return false; + } + + return true; + } + + /** + * handle show restore + */ + private ShowResultSet handleShowRestore(ConnectContext ctx, StmtExecutor executor) throws Exception { + boolean valid = validate(ctx); + if (!valid) { + throw new AnalysisException("Where clause should like: LABEL = \"your_label_name\", " + + " or LABEL LIKE \"matcher\""); + } + + Env env = ctx.getEnv(); + DatabaseIf database = ctx.getCurrentCatalog().getDbOrAnalysisException(dbName); + List<AbstractJob> jobs = env.getBackupHandler().getJobs(database.getId(), getLabelPredicate()); + List<RestoreJob> restoreJobs = jobs.stream().filter(job -> job instanceof RestoreJob) + .map(job -> (RestoreJob) job).collect(Collectors.toList()); + List<List<String>> infos; + if (needBriefResult) { + infos = restoreJobs.stream().map(RestoreJob::getBriefInfo).collect(Collectors.toList()); + } else { + infos = restoreJobs.stream().map(RestoreJob::getFullInfo).collect(Collectors.toList()); + } + + return new ShowResultSet(getMetaData(), infos); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + return handleShowRestore(ctx, executor); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitShowRestoreCommand(this, context); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 8ad7f4441b8..0844700f942 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -146,6 +146,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowQueryProfileCommand; import org.apache.doris.nereids.trees.plans.commands.ShowQueuedAnalyzeJobsCommand; import org.apache.doris.nereids.trees.plans.commands.ShowReplicaDistributionCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowRestoreCommand; import org.apache.doris.nereids.trees.plans.commands.ShowRolesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSmallFilesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; @@ -554,6 +555,10 @@ public interface CommandVisitor<R, C> { return visitCommand(showRepositoriesCommand, context); } + default R visitShowRestoreCommand(ShowRestoreCommand showRestoreCommand, C context) { + return visitCommand(showRestoreCommand, context); + } + default R visitShowRolesCommand(ShowRolesCommand showRolesCommand, C context) { return visitCommand(showRolesCommand, context); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowRestoreCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowRestoreCommandTest.java new file mode 100644 index 00000000000..2a0fdce2255 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowRestoreCommandTest.java @@ -0,0 +1,115 @@ +// 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.trees.plans.commands; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.LessThan; +import org.apache.doris.nereids.trees.expressions.Like; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; +import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.function.Predicate; + +public class ShowRestoreCommandTest extends TestWithFeService { + @Override + protected void runBeforeAll() throws Exception { + createDatabase("test"); + connectContext.setDatabase("test"); + } + + @Override + public void createTable(String sql) throws Exception { + LogicalPlan plan = new NereidsParser().parseSingle(sql); + Assertions.assertTrue(plan instanceof CreateTableCommand); + ((CreateTableCommand) plan).run(connectContext, null); + } + + @Test + void testGetLabelPredicate() throws AnalysisException, UserException { + // test where is null + ShowRestoreCommand rc = new ShowRestoreCommand("", null); + Predicate<String> re = rc.getLabelPredicate(); + Assertions.assertTrue(re.test("test")); + + // test isAccurateMatch = true + Expression where = new EqualTo(new UnboundSlot(Lists.newArrayList("label")), + new StringLiteral("mysql_edu")); + rc = new ShowRestoreCommand("test", where); + rc.validate(connectContext); // set isAccurateMatch = true + Predicate<String> re1 = rc.getLabelPredicate(); + Assertions.assertTrue(re1.test("mysql_EDU")); + + // test isAccurateMatch = false + Expression where1 = new Like(new UnboundSlot(Lists.newArrayList("label")), + new StringLiteral("mysql_edu%")); + rc = new ShowRestoreCommand("", where1); + rc.validate(connectContext); + Predicate<String> re2 = rc.getLabelPredicate(); + Assertions.assertTrue(re2.test("mysql_edu%")); + } + + @Test + void testValidate() throws UserException { + // test No database selected + ShowRestoreCommand rc = new ShowRestoreCommand("", null); + ShowRestoreCommand finalRc = rc; + connectContext.setDatabase(""); + Assertions.assertThrows(AnalysisException.class, () -> finalRc.validate(connectContext)); + connectContext.setDatabase("test"); // reset database + + // test where is null + rc = new ShowRestoreCommand("test_db", null); + Assertions.assertTrue(rc.validate(connectContext)); + + // test where is not Like and where is not EqualTo + rc = new ShowRestoreCommand("test_db", new LessThan(new UnboundSlot(Lists.newArrayList("label")), + new VarcharLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921"))); + Assertions.assertFalse(rc.validate(connectContext)); + + // test left key is not snapshotname + rc = new ShowRestoreCommand("test_db", new EqualTo(new UnboundSlot(Lists.newArrayList("notlabel")), + new VarcharLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921"))); + Assertions.assertFalse(rc.validate(connectContext)); + + // test right key is StringLikeLiteral, class is EqualTo + rc = new ShowRestoreCommand("test_db", new EqualTo(new UnboundSlot(Lists.newArrayList("label")), + new StringLiteral("mysql_edu%"))); + Assertions.assertTrue(rc.validate(connectContext)); + + // test right key is StringLikeLiteral, class is Like + rc = new ShowRestoreCommand("test_db", new Like(new UnboundSlot(Lists.newArrayList("label")), + new StringLiteral("mysql_edu%"))); + Assertions.assertTrue(rc.validate(connectContext)); + + // test right key is StringLikeLiteral but value is empty, class is Like, + rc = new ShowRestoreCommand("test_db", new Like(new UnboundSlot(Lists.newArrayList("label")), + new StringLiteral(""))); + Assertions.assertFalse(rc.validate(connectContext)); + } +} diff --git a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy index 46e9313c8db..01159117e61 100644 --- a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy +++ b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy @@ -29,6 +29,7 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th "set_replica_status," + // not a case for cloud mode, no need to run "test_be_inject_publish_txn_fail," + // not a case for cloud mode, no need to run "test_dump_image," + + "test_nereids_show_restore," + "test_index_failure_injection," + "test_information_schema_external," + "test_profile," + diff --git a/regression-test/suites/nereids_p0/show/test_nereids_show_restore.groovy b/regression-test/suites/nereids_p0/show/test_nereids_show_restore.groovy new file mode 100644 index 00000000000..4385cd8282b --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_nereids_show_restore.groovy @@ -0,0 +1,96 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_nereids_show_restore") { + String suiteName = "test_show_restore" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String dbName = "${suiteName}_db" + String tableName = "${suiteName}_table" + String snapshotName = "${suiteName}_snapshot" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0") + AGGREGATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + + List<String> values = [] + for (int i = 1; i <= 10; ++i) { + values.add("(${i}, ${i})") + } + sql "INSERT INTO ${dbName}.${tableName} VALUES ${values.join(",")}" + def result = sql "SELECT * FROM ${dbName}.${tableName}" + assertEquals(result.size(), values.size()); + + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON (${tableName}) + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + sql "TRUNCATE TABLE ${dbName}.${tableName}" + + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + ON ( `${tableName}`) + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + + result = sql "SELECT * FROM ${dbName}.${tableName}" + assertEquals(result.size(), values.size()); + checkNereidsExecute("show restore") + checkNereidsExecute("show brief restore") + checkNereidsExecute("show restore from ${dbName}") + checkNereidsExecute("show restore in ${dbName}") + checkNereidsExecute("show brief restore from ${dbName}") + checkNereidsExecute("show brief restore in ${dbName}") + checkNereidsExecute("""show restore from ${dbName} where label = "${snapshotName}" """) + checkNereidsExecute("""show restore in ${dbName} where label = "${snapshotName}" """) + checkNereidsExecute("""show brief restore from ${dbName} where label = "${snapshotName}" """) + checkNereidsExecute("""show brief restore in ${dbName} where label = "${snapshotName}" """) + checkNereidsExecute("show restore from ${dbName} where label like 'test%'") + checkNereidsExecute("show restore in ${dbName} where label like 'test%'") + checkNereidsExecute("show brief restore from ${dbName} where label like 'test%'") + checkNereidsExecute("show brief restore in ${dbName} where label like 'test%'") + + sql "DROP TABLE ${dbName}.${tableName} FORCE" + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org