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 2c0e49f0252 [Enhancement](nereids)support show backup (#48818)
2c0e49f0252 is described below
commit 2c0e49f0252321ca52074f7f7b2f4453b9088d32
Author: lsy3993 <[email protected]>
AuthorDate: Mon Mar 17 10:28:02 2025 +0800
[Enhancement](nereids)support show backup (#48818)
---
.../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/ShowBackupCommand.java | 190 +++++++++++++++++++++
.../trees/plans/visitor/CommandVisitor.java | 5 +
.../plans/commands/ShowBackupCommandTest.java | 115 +++++++++++++
.../cloud_p0/conf/regression-conf-custom.groovy | 1 +
.../show/test_nereids_show_backup.groovy | 72 ++++++++
8 files changed, 400 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 3028d099369..c2a8e4fd07f 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
@@ -275,6 +275,7 @@ supportedShowStatement
: SHOW statementScope? VARIABLES wildWhere?
#showVariables
| SHOW AUTHORS
#showAuthors
| SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier
#showCreateDatabase
+ | SHOW BACKUP ((FROM | IN) database=identifier)? wildWhere?
#showBackup
| SHOW BROKER
#showBroker
| SHOW DYNAMIC PARTITION TABLES ((FROM | IN)
database=multipartIdentifier)? #showDynamicPartition
| SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere?
#showEvents
@@ -401,7 +402,6 @@ unsupportedShowStatement
wildWhere? sortClause? limitClause?
#showPartitions
| SHOW TABLETS FROM tableName=multipartIdentifier partitionSpec?
wildWhere? sortClause? limitClause?
#showTabletsFromTable
- | SHOW BACKUP ((FROM | IN) database=multipartIdentifier)? wildWhere?
#showBackup
| SHOW BRIEF? RESTORE ((FROM | IN) database=multipartIdentifier)?
wildWhere? #showRestore
| SHOW RESOURCES wildWhere? sortClause? limitClause?
#showResources
| SHOW WORKLOAD GROUPS wildWhere?
#showWorkloadGroups
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 e1c28ba6fd9..f92fd5f391a 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
@@ -297,6 +297,7 @@ import
org.apache.doris.nereids.DorisParser.ShowAllPropertiesContext;
import org.apache.doris.nereids.DorisParser.ShowAnalyzeContext;
import org.apache.doris.nereids.DorisParser.ShowAuthorsContext;
import org.apache.doris.nereids.DorisParser.ShowBackendsContext;
+import org.apache.doris.nereids.DorisParser.ShowBackupContext;
import org.apache.doris.nereids.DorisParser.ShowBrokerContext;
import org.apache.doris.nereids.DorisParser.ShowCharsetContext;
import org.apache.doris.nereids.DorisParser.ShowCollationContext;
@@ -602,6 +603,7 @@ import
org.apache.doris.nereids.trees.plans.commands.SetUserPropertiesCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowAnalyzeCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand;
+import org.apache.doris.nereids.trees.plans.commands.ShowBackupCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCatalogCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCharsetCommand;
@@ -4969,6 +4971,19 @@ public class LogicalPlanBuilder extends
DorisParserBaseVisitor<Object> {
return new ShowBackendsCommand();
}
+ @Override
+ public LogicalPlan visitShowBackup(ShowBackupContext 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 ShowBackupCommand(dbName, wildWhere);
+ }
+
@Override
public LogicalPlan visitShowPlugins(ShowPluginsContext ctx) {
return new ShowPluginsCommand();
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 8700aa92def..fe1a5d8eb76 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
@@ -221,6 +221,7 @@ public enum PlanType {
SHOW_ANALYZE_COMMAND,
SHOW_QUEUED_ANALYZE_JOBS_COMMAND,
SHOW_BACKENDS_COMMAND,
+ SHOW_BACKUP_COMMAND,
SHOW_BLOCK_RULE_COMMAND,
SHOW_BROKER_COMMAND,
SHOW_CATALOG_COMMAND,
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBackupCommand.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBackupCommand.java
new file mode 100644
index 00000000000..18c13827c36
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBackupCommand.java
@@ -0,0 +1,190 @@
+// 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.BackupJob;
+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 backup command
+ */
+public class ShowBackupCommand extends ShowCommand {
+ public static final ImmutableList<String> TITLE_NAMES = new
ImmutableList.Builder<String>()
+
.add("JobId").add("SnapshotName").add("DbName").add("State").add("BackupObjs").add("CreateTime")
+
.add("SnapshotFinishedTime").add("UploadFinishedTime").add("FinishedTime").add("UnfinishedTasks")
+ .add("Progress").add("TaskErrMsg").add("Status").add("Timeout")
+ .build();
+
+ private String dbName;
+ private Expression where;
+ private boolean isAccurateMatch;
+ private String snapshotName;
+
+ /**
+ * constructor
+ */
+ public ShowBackupCommand(String dbName, Expression where) {
+ super(PlanType.SHOW_BACKUP_COMMAND);
+ this.dbName = dbName;
+ this.where = where;
+ }
+
+ /**
+ * get metadata
+ */
+ private ShowResultSetMetaData getMetaData() {
+ ShowResultSetMetaData.Builder builder =
ShowResultSetMetaData.builder();
+ for (String title : TITLE_NAMES) {
+ builder.addColumn(new Column(title, ScalarType.STRING));
+ }
+ return builder.build();
+ }
+
+ /**
+ * get label predicate for show backup
+ */
+ @VisibleForTesting
+ protected Predicate<String> getSnapshotPredicate() throws
AnalysisException {
+ if (null == where) {
+ return label -> true;
+ }
+ if (isAccurateMatch) {
+ return CaseSensibility.LABEL.getCaseSensibility()
+ ? label -> label.equals(snapshotName) : label ->
label.equalsIgnoreCase(snapshotName);
+ } else {
+ PatternMatcher patternMatcher =
PatternMatcherWrapper.createMysqlPattern(
+ snapshotName, 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 backup 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 (!"snapshotname".equalsIgnoreCase(leftKey)) {
+ return false;
+ }
+
+ // right child
+ if (!(where.child(1) instanceof StringLikeLiteral)) {
+ return false;
+ }
+ snapshotName = ((StringLikeLiteral) where.child(1)).getStringValue();
+ if (Strings.isNullOrEmpty(snapshotName)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ /**
+ * handle show backup
+ */
+ private ShowResultSet handleShowBackup(ConnectContext ctx, StmtExecutor
executor) throws Exception {
+ boolean valid = validate(ctx);
+ if (!valid) {
+ throw new AnalysisException("Where clause should like:
SnapshotName = \"your_snapshot_name\", "
+ + " or SnapshotName LIKE \"matcher\"");
+ }
+
+ DatabaseIf database =
ctx.getCurrentCatalog().getDbOrAnalysisException(dbName);
+ List<AbstractJob> jobs = Env.getCurrentEnv().getBackupHandler()
+ .getJobs(database.getId(), getSnapshotPredicate());
+ List<BackupJob> backupJobs = jobs.stream().filter(job -> job
instanceof BackupJob)
+ .map(job -> (BackupJob) job).collect(Collectors.toList());
+ List<List<String>> infos =
backupJobs.stream().map(BackupJob::getInfo).collect(Collectors.toList());
+
+ return new ShowResultSet(getMetaData(), infos);
+ }
+
+ @Override
+ public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor)
throws Exception {
+ return handleShowBackup(ctx, executor);
+ }
+
+ @Override
+ public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+ return visitor.visitShowBackupCommand(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 d4fec71f0be..01ed6315fa3 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
@@ -102,6 +102,7 @@ import
org.apache.doris.nereids.trees.plans.commands.SetUserPropertiesCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowAnalyzeCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowAuthorsCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowBackendsCommand;
+import org.apache.doris.nereids.trees.plans.commands.ShowBackupCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowBrokerCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCatalogCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCharsetCommand;
@@ -512,6 +513,10 @@ public interface CommandVisitor<R, C> {
return visitCommand(showBackendsCommand, context);
}
+ default R visitShowBackupCommand(ShowBackupCommand showBackupCommand, C
context) {
+ return visitCommand(showBackupCommand, context);
+ }
+
default R visitShowCreateTableCommand(ShowCreateTableCommand
showCreateTableCommand, C context) {
return visitCommand(showCreateTableCommand, context);
}
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowBackupCommandTest.java
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowBackupCommandTest.java
new file mode 100644
index 00000000000..ce75598a38b
--- /dev/null
+++
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowBackupCommandTest.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 ShowBackupCommandTest 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 testGetSnapshotPredicate() throws AnalysisException, UserException {
+ // test where is null
+ ShowBackupCommand bc = new ShowBackupCommand("", null);
+ Predicate<String> re = bc.getSnapshotPredicate();
+ Assertions.assertTrue(re.test("test"));
+
+ // test isAccurateMatch = true
+ Expression where = new EqualTo(new
UnboundSlot(Lists.newArrayList("snapshotname")),
+ new StringLiteral("mysql_edu"));
+ bc = new ShowBackupCommand("test", where);
+ bc.validate(connectContext); // set isAccurateMatch = true
+ Predicate<String> re1 = bc.getSnapshotPredicate();
+ Assertions.assertTrue(re1.test("mysql_EDU"));
+
+ // test isAccurateMatch = false
+ Expression where1 = new Like(new
UnboundSlot(Lists.newArrayList("snapshotname")),
+ new StringLiteral("mysql_edu%"));
+ bc = new ShowBackupCommand("", where1);
+ bc.validate(connectContext);
+ Predicate<String> re2 = bc.getSnapshotPredicate();
+ Assertions.assertTrue(re2.test("mysql_edu%"));
+ }
+
+ @Test
+ void testValidate() throws UserException {
+ // test No database selected
+ ShowBackupCommand bc = new ShowBackupCommand("", null);
+ ShowBackupCommand finalBc = bc;
+ connectContext.setDatabase("");
+ Assertions.assertThrows(AnalysisException.class, () ->
finalBc.validate(connectContext));
+ connectContext.setDatabase("test"); // reset database
+
+ // test where is null
+ bc = new ShowBackupCommand("test_db", null);
+ Assertions.assertTrue(bc.validate(connectContext));
+
+ // test where is not Like and where is not EqualTo
+ bc = new ShowBackupCommand("test_db", new LessThan(new
UnboundSlot(Lists.newArrayList("snapshotname")),
+ new
VarcharLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921")));
+ Assertions.assertFalse(bc.validate(connectContext));
+
+ // test left key is not snapshotname
+ bc = new ShowBackupCommand("test_db", new EqualTo(new
UnboundSlot(Lists.newArrayList("notsnapshotname")),
+ new
VarcharLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921")));
+ Assertions.assertFalse(bc.validate(connectContext));
+
+ // test right key is StringLikeLiteral, class is EqualTo
+ bc = new ShowBackupCommand("test_db", new EqualTo(new
UnboundSlot(Lists.newArrayList("snapshotname")),
+ new StringLiteral("mysql_edu%")));
+ Assertions.assertTrue(bc.validate(connectContext));
+
+ // test right key is StringLikeLiteral, class is Like
+ bc = new ShowBackupCommand("test_db", new Like(new
UnboundSlot(Lists.newArrayList("snapshotname")),
+ new StringLiteral("mysql_edu%")));
+ Assertions.assertTrue(bc.validate(connectContext));
+
+ // test right key is StringLikeLiteral but value is empty, class is
Like,
+ bc = new ShowBackupCommand("test_db", new Like(new
UnboundSlot(Lists.newArrayList("snapshotname")),
+ new StringLiteral("")));
+ Assertions.assertFalse(bc.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 e3f6b7e6f61..46e9313c8db 100644
--- a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy
+++ b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy
@@ -47,6 +47,7 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + //
keep this line as th
"test_topn_fault_injection," +
"auto_partition_in_partition_prune," + // inserted data in too many
tablets, txn to large. not suitable for cloud.
"one_col_range_partition," + // inserted data in too many tablets, txn to
large. not suitable for cloud.
+ "test_nereids_show_backup," +
"zzz_the_end_sentinel_do_not_touch" // keep this line as the last line
excludeDirectories = "000_the_start_sentinel_do_not_touch," + // keep this
line as the first line
diff --git
a/regression-test/suites/nereids_p0/show/test_nereids_show_backup.groovy
b/regression-test/suites/nereids_p0/show/test_nereids_show_backup.groovy
new file mode 100644
index 00000000000..f0551dd1a97
--- /dev/null
+++ b/regression-test/suites/nereids_p0/show/test_nereids_show_backup.groovy
@@ -0,0 +1,72 @@
+// 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_backup") {
+ String suiteName = "test_show_backup"
+ 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)
+
+ checkNereidsExecute("show backup")
+ checkNereidsExecute("show backup from ${dbName}")
+ checkNereidsExecute("show backup in ${dbName}")
+ checkNereidsExecute("""show backup from ${dbName} where SnapshotName =
"${snapshotName}" """)
+ checkNereidsExecute("""show backup in ${dbName} where SnapshotName =
"${snapshotName}" """)
+ checkNereidsExecute("show backup from ${dbName} where SnapshotName like
'test%'")
+ checkNereidsExecute("show backup in ${dbName} where SnapshotName like
'test%'")
+
+ sql "DROP TABLE ${dbName}.${tableName} FORCE"
+ sql "DROP DATABASE ${dbName} FORCE"
+ sql "DROP REPOSITORY `${repoName}`"
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]