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

Reply via email to