This is an automated email from the ASF dual-hosted git repository. englefly 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 a078defffbd [Enhancement](nereids)support show snapshot (#49274) a078defffbd is described below commit a078defffbd1bc5ea452f406dffd99a888e7723e Author: lsy3993 <110876560+lsy3...@users.noreply.github.com> AuthorDate: Tue Mar 25 09:37:23 2025 +0800 [Enhancement](nereids)support show snapshot (#49274) support show snapshot in nereids --- .../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/ShowSnapshotCommand.java | 225 +++++++++++++++++++++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../plans/commands/ShowSnapshotCommandTest.java | 126 ++++++++++++ .../cloud_p0/conf/regression-conf-custom.groovy | 1 + .../show/test_nereids_show_snapshot.groovy | 71 +++++++ 8 files changed, 445 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 5be8d123784..768741cd6d9 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 @@ -283,6 +283,7 @@ supportedShowStatement | SHOW ALL? GRANTS #showGrants | SHOW GRANTS FOR userIdentify #showGrantsForUser | SHOW SYNC JOB ((FROM | IN) database=multipartIdentifier)? #showSyncJob + | SHOW SNAPSHOT ON repo=identifier wildWhere? #showSnapshot | SHOW LOAD PROFILE loadIdPath=STRING_LITERAL? limitClause? #showLoadProfile | SHOW CREATE REPOSITORY FOR identifier #showCreateRepository | SHOW VIEW @@ -403,7 +404,6 @@ unsupportedShowStatement wildWhere? sortClause? limitClause? #showTabletsFromTable | SHOW RESOURCES wildWhere? sortClause? limitClause? #showResources | SHOW WORKLOAD GROUPS wildWhere? #showWorkloadGroups - | SHOW SNAPSHOT ON repo=identifier wildWhere? #showSnapshot | SHOW FULL? BUILTIN? FUNCTIONS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showFunctions | SHOW GLOBAL FULL? FUNCTIONS wildWhere? #showGlobalFunctions 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 82f5fb84589..8a918144aa4 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 @@ -339,6 +339,7 @@ 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.ShowSnapshotContext; import org.apache.doris.nereids.DorisParser.ShowSqlBlockRuleContext; import org.apache.doris.nereids.DorisParser.ShowStagesContext; import org.apache.doris.nereids.DorisParser.ShowStatusContext; @@ -652,6 +653,7 @@ 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.ShowSnapshotCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStagesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStatusCommand; @@ -5007,6 +5009,19 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { return new ShowSmallFilesCommand(dbName); } + @Override + public LogicalPlan visitShowSnapshot(ShowSnapshotContext ctx) { + String repoName = null; + Expression wildWhere = null; + if (ctx.wildWhere() != null) { + wildWhere = getWildWhere(ctx.wildWhere()); + } + if (ctx.repo != null) { + repoName = ctx.repo.getText(); + } + return new ShowSnapshotCommand(repoName, wildWhere); + } + @Override public LogicalPlan visitShowSqlBlockRule(ShowSqlBlockRuleContext ctx) { String ruleName = null; 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 a56a79d9777..15f356e9722 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 @@ -261,6 +261,7 @@ public enum PlanType { SHOW_RESTORE_COMMAND, SHOW_ROLE_COMMAND, SHOW_SMALL_FILES_COMMAND, + SHOW_SNAPSHOT_COMMAND, SHOW_STAGES_COMMAND, SHOW_STATUS_COMMAND, SHOW_STORAGE_ENGINES_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSnapshotCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSnapshotCommand.java new file mode 100644 index 00000000000..b4dd9b74beb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowSnapshotCommand.java @@ -0,0 +1,225 @@ +// 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.Repository; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; +import org.apache.doris.nereids.trees.expressions.Expression; +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; + +/** + * show snapshot command + */ +public class ShowSnapshotCommand extends ShowCommand { + /** + * only remote and local type + */ + public enum SnapshotType { + REMOTE, + LOCAL + } + + public static final ImmutableList<String> SNAPSHOT_ALL = new ImmutableList.Builder<String>() + .add("Snapshot").add("Timestamp").add("Status") + .build(); + public static final ImmutableList<String> SNAPSHOT_DETAIL = new ImmutableList.Builder<String>() + .add("Snapshot").add("Timestamp").add("Database").add("Details").add("Status") + .build(); + + private String repoName; + private Expression where; + private String snapshotName; + private String timestamp; + private SnapshotType snapshotType = SnapshotType.REMOTE; + + /** + * constructor + */ + public ShowSnapshotCommand(String repoName, Expression where) { + super(PlanType.SHOW_SNAPSHOT_COMMAND); + this.repoName = repoName; + this.where = where; + } + + public String getRepoName() { + return repoName; + } + + public String getSnapshotName() { + return snapshotName; + } + + public String getTimestamp() { + return timestamp; + } + + public String getSnapshotType() { + return snapshotType.name(); + } + + private ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + if (!Strings.isNullOrEmpty(snapshotName) && !Strings.isNullOrEmpty(timestamp)) { + for (String title : SNAPSHOT_DETAIL) { + builder.addColumn(new Column(title, ScalarType.STRING)); + } + } else { + for (String title : SNAPSHOT_ALL) { + builder.addColumn(new Column(title, ScalarType.STRING)); + } + } + return builder.build(); + } + + private boolean analyzeSubExpression(ComparisonPredicate expr) { + Expression key = expr.child(0); + Expression val = expr.child(1); + + if (!(key instanceof UnboundSlot)) { + return false; + } + if (!(val instanceof StringLikeLiteral)) { + return false; + } + + String name = ((UnboundSlot) key).getName(); + if (name.equalsIgnoreCase("snapshot")) { + snapshotName = ((StringLikeLiteral) val).getStringValue(); + if (Strings.isNullOrEmpty(snapshotName)) { + return false; + } + return true; + } else if (name.equalsIgnoreCase("timestamp")) { + timestamp = ((StringLikeLiteral) val).getStringValue(); + if (Strings.isNullOrEmpty(timestamp)) { + return false; + } + return true; + } else if (name.equalsIgnoreCase("snapshotType")) { + String snapshotTypeVal = ((StringLikeLiteral) val).getStringValue(); + if (Strings.isNullOrEmpty(snapshotTypeVal)) { + return false; + } + // snapshotType now only support "remote" and "local" + switch (snapshotTypeVal.toLowerCase()) { + case "remote": + snapshotType = SnapshotType.REMOTE; + return true; + case "local": + snapshotType = SnapshotType.LOCAL; + return true; + default: + return false; + } + } else { + return false; + } + } + + /** + * validate + */ + @VisibleForTesting + protected boolean validate(ConnectContext ctx) throws UserException { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } + + // validate analyze where clause if not null + boolean ok = true; + if (where != null) { + CHECK: { + if (where instanceof ComparisonPredicate) { + if (!analyzeSubExpression((ComparisonPredicate) where)) { + ok = false; + } + } else if (where instanceof And) { + if (!(where.child(0) instanceof ComparisonPredicate) + || !(where.child(1) instanceof ComparisonPredicate)) { + ok = false; + break CHECK; + } + if (!analyzeSubExpression((ComparisonPredicate) where.child(0)) + || !analyzeSubExpression((ComparisonPredicate) where.child(1))) { + ok = false; + } + } else { + ok = false; + } + } + + if (ok && (Strings.isNullOrEmpty(snapshotName) && !Strings.isNullOrEmpty(timestamp))) { + // can not only set timestamp + ok = false; + } + } + return ok; + } + + /** + * handle show backup + */ + private ShowResultSet handleShowSnapshot(ConnectContext ctx, StmtExecutor executor) throws Exception { + boolean valid = validate(ctx); + if (!valid) { + throw new AnalysisException("Where clause should looks like: SNAPSHOT = 'your_snapshot_name'" + + " [AND TIMESTAMP = '2018-04-18-19-19-10'] [AND SNAPSHOTTYPE = 'remote' | 'local']"); + } + + Repository repo = Env.getCurrentEnv().getBackupHandler().getRepoMgr().getRepo(repoName); + if (repo == null) { + throw new AnalysisException("Repository " + repoName + " does not exist"); + } + List<List<String>> snapshotInfos = repo.getSnapshotInfos(snapshotName, timestamp); + return new ShowResultSet(getMetaData(), snapshotInfos); + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + return handleShowSnapshot(ctx, executor); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitShowSnapshotCommand(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 14d294763fc..35447960f33 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 @@ -150,6 +150,7 @@ 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.ShowSnapshotCommand; import org.apache.doris.nereids.trees.plans.commands.ShowSqlBlockRuleCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStagesCommand; import org.apache.doris.nereids.trees.plans.commands.ShowStatusCommand; @@ -532,6 +533,10 @@ public interface CommandVisitor<R, C> { return visitCommand(showSmallFilesCommand, context); } + default R visitShowSnapshotCommand(ShowSnapshotCommand showSnapshotCommand, C context) { + return visitCommand(showSnapshotCommand, context); + } + default R visitShowSqlBlockRuleCommand(ShowSqlBlockRuleCommand showblockruleCommand, C context) { return visitCommand(showblockruleCommand, context); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowSnapshotCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowSnapshotCommandTest.java new file mode 100644 index 00000000000..e9d2d7147ea --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowSnapshotCommandTest.java @@ -0,0 +1,126 @@ +// 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.UserException; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.expressions.And; +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.Or; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; +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; + +public class ShowSnapshotCommandTest 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 testValidate() throws UserException { + // test where is null + ShowSnapshotCommand ss = new ShowSnapshotCommand("test", null); + Assertions.assertTrue(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot and child(1) is StringLiteral + Expression where1 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshot")), + new StringLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921")); + ss = new ShowSnapshotCommand("test", where1); + Assertions.assertTrue(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot but value is empty + Expression where2 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshot")), + new StringLiteral("")); + ss = new ShowSnapshotCommand("test", where2); + Assertions.assertFalse(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot but key is timestamp. It should return false + Expression where3 = new EqualTo(new UnboundSlot(Lists.newArrayList("timestamp")), + new StringLiteral("2025-01-01 01:01:01")); + ss = new ShowSnapshotCommand("test", where3); + Assertions.assertFalse(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot but value is empty + Expression where4 = new EqualTo(new UnboundSlot(Lists.newArrayList("timestamp")), + new StringLiteral("")); + ss = new ShowSnapshotCommand("test", where4); + Assertions.assertFalse(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot but key is snapshotType + Expression where5 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshotType")), + new StringLiteral("remote")); + ss = new ShowSnapshotCommand("test", where5); + Assertions.assertTrue(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot but value is empty + Expression where6 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshotType")), + new StringLiteral("")); + ss = new ShowSnapshotCommand("test", where6); + Assertions.assertFalse(ss.validate(connectContext)); + + // test where is ComparisonPredicate, child(0) is UnboundSlot, key is snapshotType but value is other word + Expression where7 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshotType")), + new StringLiteral("xxx")); + ss = new ShowSnapshotCommand("test", where7); + Assertions.assertFalse(ss.validate(connectContext)); + + // test where is And, child(0) and child(1) both ComparisonPredicate + Expression equalTo1 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshot")), + new StringLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921")); + Expression equalTo2 = new EqualTo(new UnboundSlot(Lists.newArrayList("timestamp")), + new StringLiteral("2018-04-18-19-19-10")); + Expression where8 = new And(equalTo1, equalTo2); + ss = new ShowSnapshotCommand("test", where8); + Assertions.assertTrue(ss.validate(connectContext)); + + // test where is And, child(0) or child(1) is ComparisonPredicate + Expression equalTo3 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshot")), + new StringLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921")); + Expression like = new Like(new UnboundSlot(Lists.newArrayList("timestamp")), + new StringLiteral("2018-04-18-19-19-10%")); + Expression where9 = new And(equalTo3, like); + ss = new ShowSnapshotCommand("test", where9); + Assertions.assertFalse(ss.validate(connectContext)); + + // test where is Or + Expression equalTo4 = new EqualTo(new UnboundSlot(Lists.newArrayList("snapshot")), + new StringLiteral("mysql_edu_mall_mall_source_occupy_1741669209_174166921")); + Expression equalTo5 = new EqualTo(new UnboundSlot(Lists.newArrayList("timestamp")), + new StringLiteral("2018-04-18-19-19-10")); + Expression where10 = new Or(equalTo4, equalTo5); + ss = new ShowSnapshotCommand("test", where10); + Assertions.assertFalse(ss.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 01159117e61..2ce6002462e 100644 --- a/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy +++ b/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy @@ -44,6 +44,7 @@ excludeSuites = "000_the_start_sentinel_do_not_touch," + // keep this line as th "test_query_sys_rowsets," + // rowsets sys table "test_unique_table_debug_data," + // disable auto compaction "test_insert," + // txn insert + "test_nereids_show_snapshot," + "test_full_compaction_run_status," + "test_topn_fault_injection," + "auto_partition_in_partition_prune," + // inserted data in too many tablets, txn to large. not suitable for cloud. diff --git a/regression-test/suites/nereids_p0/show/test_nereids_show_snapshot.groovy b/regression-test/suites/nereids_p0/show/test_nereids_show_snapshot.groovy new file mode 100644 index 00000000000..096ed58fa2d --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_nereids_show_snapshot.groovy @@ -0,0 +1,71 @@ +// 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_snapshot") { + String suiteName = "test_show_snapshot" + 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 snapshot on ${repoName}") + checkNereidsExecute("""show snapshot on ${repoName} where SNAPSHOT = "${snapshotName}" """) + checkNereidsExecute("""show snapshot on ${repoName} where SNAPSHOT = "${snapshotName}" + and TIMESTAMP > '2025-01-01' """) + checkNereidsExecute("""show snapshot on ${repoName} where SNAPSHOT = "${snapshotName}" + and SNAPSHOTTYPE = 'remote'""") + + 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