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 078c0e2599e [Enhancement](nereids)support show export (#51360)
078c0e2599e is described below
commit 078c0e2599eec772e8cb3c03cc29e5f68db5c0af
Author: lsy3993 <[email protected]>
AuthorDate: Fri May 30 15:33:48 2025 +0800
[Enhancement](nereids)support show export (#51360)
---
.../antlr4/org/apache/doris/nereids/DorisParser.g4 | 4 +-
.../doris/nereids/parser/LogicalPlanBuilder.java | 37 ++++
.../apache/doris/nereids/trees/plans/PlanType.java | 1 +
.../trees/plans/commands/ShowExportCommand.java | 218 +++++++++++++++++++++
.../trees/plans/visitor/CommandVisitor.java | 5 +
.../plans/commands/ShowExportCommandTest.java | 106 ++++++++++
.../show/test_nereids_show_export.groovy | 166 ++++++++++++++++
7 files changed, 535 insertions(+), 2 deletions(-)
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 5eca5522832..ea59fda6c99 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
@@ -320,6 +320,8 @@ supportedShowStatement
| SHOW BROKER
#showBroker
| SHOW DYNAMIC PARTITION TABLES ((FROM | IN)
database=multipartIdentifier)? #showDynamicPartition
| SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere?
#showEvents
+ | SHOW EXPORT ((FROM | IN) database=multipartIdentifier)? wildWhere?
+ sortClause? limitClause?
#showExport
| SHOW LAST INSERT
#showLastInsert
| SHOW ((CHAR SET) | CHARSET)
#showCharset
| SHOW DELETE ((FROM | IN) database=multipartIdentifier)?
#showDelete
@@ -473,8 +475,6 @@ unsupportedShowStatement
((FROM | IN) database=multipartIdentifier)?
#showCreateFunction
| SHOW LOAD WARNINGS ((((FROM | IN) database=multipartIdentifier)?
wildWhere? limitClause?) | (ON url=STRING_LITERAL))
#showLoadWarings
- | SHOW EXPORT ((FROM | IN) database=multipartIdentifier)? wildWhere?
- sortClause? limitClause?
#showExport
| SHOW ALTER TABLE (ROLLUP | (MATERIALIZED VIEW) | COLUMN)
((FROM | IN) database=multipartIdentifier)? wildWhere?
sortClause? limitClause?
#showAlterTable
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 cd4425ccad0..54ef733c109 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
@@ -352,6 +352,7 @@ import
org.apache.doris.nereids.DorisParser.ShowDictionariesContext;
import org.apache.doris.nereids.DorisParser.ShowDynamicPartitionContext;
import org.apache.doris.nereids.DorisParser.ShowEncryptKeysContext;
import org.apache.doris.nereids.DorisParser.ShowEventsContext;
+import org.apache.doris.nereids.DorisParser.ShowExportContext;
import org.apache.doris.nereids.DorisParser.ShowFrontendsContext;
import org.apache.doris.nereids.DorisParser.ShowFunctionsContext;
import org.apache.doris.nereids.DorisParser.ShowGlobalFunctionsContext;
@@ -718,6 +719,7 @@ import
org.apache.doris.nereids.trees.plans.commands.ShowDictionariesCommand;
import
org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowEncryptKeysCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand;
+import org.apache.doris.nereids.trees.plans.commands.ShowExportCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowFunctionsCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand;
@@ -4629,6 +4631,41 @@ public class LogicalPlanBuilder extends
DorisParserBaseVisitor<Object> {
return new ShowEventsCommand();
}
+ @Override
+ public LogicalPlan visitShowExport(ShowExportContext ctx) {
+ String ctlName = null;
+ String dbName = null;
+ Expression wildWhere = null;
+ List<OrderKey> orderKeys = null;
+ long limit = -1L;
+ if (ctx.database != null) {
+ List<String> nameParts = visitMultipartIdentifier(ctx.database);
+ if (nameParts.size() == 1) {
+ dbName = nameParts.get(0);
+ } else if (nameParts.size() == 2) {
+ ctlName = nameParts.get(0);
+ dbName = nameParts.get(1);
+ } else {
+ throw new AnalysisException("nameParts in analyze database
should be [ctl.]db");
+ }
+ }
+ if (ctx.sortClause() != null) {
+ orderKeys = visit(ctx.sortClause().sortItem(), OrderKey.class);
+ }
+ if (ctx.wildWhere() != null) {
+ wildWhere = getWildWhere(ctx.wildWhere());
+ }
+ if (ctx.limitClause() != null) {
+ limit = ctx.limitClause().limit != null
+ ? Long.parseLong(ctx.limitClause().limit.getText())
+ : 0;
+ if (limit < 0) {
+ throw new ParseException("Limit requires non-negative number",
ctx.limitClause());
+ }
+ }
+ return new ShowExportCommand(ctlName, dbName, wildWhere, orderKeys,
limit);
+ }
+
@Override
public LogicalPlan visitShowConfig(ShowConfigContext ctx) {
ShowConfigCommand command;
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 e2ac287fe60..b317e9b4c45 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
@@ -267,6 +267,7 @@ public enum PlanType {
SHOW_DYNAMIC_PARTITION_COMMAND,
SHOW_ENCRYPT_KEYS_COMMAND,
SHOW_EVENTS_COMMAND,
+ SHOW_EXPORT_COMMAND,
SHOW_DATA_TYPES_COMMAND,
SHOW_FRONTENDS_COMMAND,
SHOW_FUNCTIONS_COMMAND,
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowExportCommand.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowExportCommand.java
new file mode 100644
index 00000000000..f9fe901a997
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowExportCommand.java
@@ -0,0 +1,218 @@
+// 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.analysis.RedirectStatus;
+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.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.UserException;
+import org.apache.doris.common.util.OrderByPair;
+import org.apache.doris.datasource.CatalogIf;
+import org.apache.doris.load.ExportJobState;
+import org.apache.doris.load.ExportMgr;
+import org.apache.doris.nereids.analyzer.UnboundSlot;
+import org.apache.doris.nereids.properties.OrderKey;
+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.IntegerLikeLiteral;
+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 com.google.common.collect.Sets;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * show export command
+ */
+public class ShowExportCommand extends ShowCommand {
+
+ private static final ImmutableList<String> TITLE_NAMES = new
ImmutableList.Builder<String>()
+ .add("JobId").add("Label").add("State").add("Progress")
+ .add("TaskInfo").add("Path")
+ .add("CreateTime").add("StartTime").add("FinishTime")
+ .add("Timeout").add("ErrorMsg").add("OutfileInfo")
+ .build();
+ private static final String ID = "id";
+ private static final String STATE = "state";
+ private static final String LABEL = "label";
+ private static final Logger LOG =
LogManager.getLogger(ShowExportCommand.class);
+ private String ctl;
+ private String dbName;
+ private final Expression wildWhere;
+ private final long limit;
+ private final List<OrderKey> orderKeys;
+ private ArrayList<OrderByPair> orderByPairs;
+ private String stateValue = null;
+ private boolean isLabelUseLike = false;
+ private ExportJobState jobState;
+ private long jobId = 0;
+ private String label = null;
+
+ /**
+ * constructor for show export
+ */
+ public ShowExportCommand(String ctl, String dbName, Expression wildWhere,
List<OrderKey> orderKeys, long limit) {
+ super(PlanType.SHOW_EXPORT_COMMAND);
+ this.ctl = ctl;
+ this.dbName = dbName;
+ this.wildWhere = wildWhere;
+ this.orderKeys = orderKeys;
+ this.limit = limit;
+ }
+
+ @Override
+ public ShowResultSetMetaData getMetaData() {
+ ShowResultSetMetaData.Builder builder =
ShowResultSetMetaData.builder();
+ for (String title : TITLE_NAMES) {
+ builder.addColumn(new Column(title,
ScalarType.createVarchar(128)));
+ }
+ return builder.build();
+ }
+
+ private ExportJobState getJobState() {
+ if (Strings.isNullOrEmpty(stateValue)) {
+ return null;
+ }
+ return jobState;
+ }
+
+ @VisibleForTesting
+ protected void validate(ConnectContext ctx) throws UserException {
+ if (Strings.isNullOrEmpty(ctl)) {
+ ctl = ctx.getDefaultCatalog();
+ if (Strings.isNullOrEmpty(ctl)) {
+
ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_CATALOG);
+ }
+ }
+
+ if (Strings.isNullOrEmpty(dbName)) {
+ dbName = ctx.getDatabase();
+ if (Strings.isNullOrEmpty(dbName)) {
+ ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
+ }
+ }
+
+ // where
+ if (wildWhere != null) {
+ analyzeExpression(wildWhere);
+ }
+ }
+
+ private void analyzeExpression(Expression expr) throws AnalysisException {
+ if (expr == null) {
+ return;
+ }
+ boolean valid = false;
+
+ if (expr.child(0) instanceof UnboundSlot) {
+ String leftKey = ((UnboundSlot)
expr.child(0)).getName().toLowerCase();
+
+ if (expr instanceof EqualTo) {
+ if (ID.equalsIgnoreCase(leftKey) && expr.child(1) instanceof
IntegerLikeLiteral) {
+ jobId = ((IntegerLikeLiteral)
expr.child(1)).getLongValue();
+ valid = true;
+ } else if (STATE.equalsIgnoreCase(leftKey) && expr.child(1)
instanceof StringLikeLiteral) {
+ String value = ((StringLikeLiteral)
expr.child(1)).getStringValue();
+ if (!Strings.isNullOrEmpty(value)) {
+ stateValue = value.toUpperCase();
+ try {
+ jobState = ExportJobState.valueOf(stateValue);
+ valid = true;
+ } catch (IllegalArgumentException e) {
+ LOG.warn("illegal state argument in export stmt.
stateValue={}, error={}", stateValue, e);
+ }
+ }
+ } else if (LABEL.equalsIgnoreCase(leftKey) && expr.child(1)
instanceof StringLikeLiteral) {
+ label = ((StringLikeLiteral)
expr.child(1)).getStringValue();
+ valid = true;
+ }
+ } else if (expr instanceof Like) {
+ if (LABEL.equalsIgnoreCase(leftKey) && expr.child(1)
instanceof StringLikeLiteral) {
+ label = ((StringLikeLiteral)
expr.child(1)).getStringValue();
+ isLabelUseLike = true;
+ valid = true;
+ }
+ }
+ }
+
+ if (!valid) {
+ throw new AnalysisException("Where clause should looks like below:
"
+ + " ID = $your_job_id, or STATE =
\"PENDING|EXPORTING|FINISHED|CANCELLED\", "
+ + "or LABEL = \"xxx\" or LABEL like \"xxx%\"");
+ }
+ }
+
+ private ShowResultSet handleShowExport(ConnectContext ctx, StmtExecutor
executor) throws Exception {
+ // first validate the where
+ validate(ctx);
+
+ // then process the order by
+ orderByPairs = getOrderByPairs(orderKeys, TITLE_NAMES);
+
+ // last get export info
+ Env env = Env.getCurrentEnv();
+ CatalogIf catalog =
env.getCatalogMgr().getCatalogOrAnalysisException(ctl);
+ DatabaseIf db = catalog.getDbOrAnalysisException(dbName);
+ long dbId = db.getId();
+
+ ExportMgr exportMgr = env.getExportMgr();
+
+ Set<ExportJobState> states = null;
+ ExportJobState state = getJobState();
+ if (state != null) {
+ states = Sets.newHashSet(state);
+ }
+ List<List<String>> infos = exportMgr.getExportJobInfosByIdOrState(
+ dbId, jobId, label, isLabelUseLike, states, orderByPairs,
limit >= 0 ? limit : -1);
+ return new ShowResultSet(getMetaData(), infos);
+ }
+
+ @Override
+ public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor)
throws Exception {
+ return handleShowExport(ctx, executor);
+ }
+
+ @Override
+ public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+ return visitor.visitShowExportCommand(this, context);
+ }
+
+ @Override
+ public RedirectStatus toRedirectStatus() {
+ return RedirectStatus.FORWARD_NO_SYNC;
+ }
+}
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 0804781964f..0004dc7f9a5 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
@@ -174,6 +174,7 @@ import
org.apache.doris.nereids.trees.plans.commands.ShowDictionariesCommand;
import
org.apache.doris.nereids.trees.plans.commands.ShowDynamicPartitionCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowEncryptKeysCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowEventsCommand;
+import org.apache.doris.nereids.trees.plans.commands.ShowExportCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowFrontendsCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowFunctionsCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowGrantsCommand;
@@ -909,6 +910,10 @@ public interface CommandVisitor<R, C> {
return visitCommand(showEventsCommand, context);
}
+ default R visitShowExportCommand(ShowExportCommand showExportCommand, C
context) {
+ return visitCommand(showExportCommand, context);
+ }
+
default R visitShowDeleteCommand(ShowDeleteCommand showDeleteCommand, C
context) {
return visitCommand(showDeleteCommand, context);
}
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowExportCommandTest.java
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowExportCommandTest.java
new file mode 100644
index 00000000000..fc7329eefcb
--- /dev/null
+++
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowExportCommandTest.java
@@ -0,0 +1,106 @@
+// 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.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.literal.IntegerLiteral;
+import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
+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 ShowExportCommandTest extends TestWithFeService {
+ @Test
+ void testValidate() throws UserException {
+ ShowExportCommand se = new ShowExportCommand("internal", "test", null,
null, -1);
+ se.validate(connectContext);
+
+ se = new ShowExportCommand("", "test", null, null, -1);
+ se.validate(connectContext);
+
+ se = new ShowExportCommand("", "test", null, null, -1);
+ se.validate(connectContext);
+
+ Expression where1 = new EqualTo(new
UnboundSlot(Lists.newArrayList("LABEL")),
+ new StringLiteral("xxx"));
+ se = new ShowExportCommand("", "test", where1, null, -1);
+ se.validate(connectContext);
+
+ Expression where2 = new EqualTo(new
UnboundSlot(Lists.newArrayList("id")),
+ new IntegerLiteral(123));
+ se = new ShowExportCommand("", "test", where2, null, -1);
+ se.validate(connectContext);
+
+ Expression where3 = new EqualTo(new
UnboundSlot(Lists.newArrayList("state")),
+ new StringLiteral("FINISHED"));
+ se = new ShowExportCommand("", "test", where3, null, -1);
+ se.validate(connectContext);
+
+ Expression where4 = new Like(new
UnboundSlot(Lists.newArrayList("LABEL")),
+ new StringLiteral("xx%"));
+ se = new ShowExportCommand("", "test", where4, null, -1);
+ se.validate(connectContext);
+
+ Expression where5 = new EqualTo(new
UnboundSlot(Lists.newArrayList("state1")),
+ new StringLiteral("xxx"));
+ se = new ShowExportCommand("", "test", where5, null, -1);
+ ShowExportCommand finalSe1 = se;
+ Assertions.assertThrows(AnalysisException.class, () ->
finalSe1.validate(connectContext));
+
+ Expression where6 = new Like(new UnboundSlot(Lists.newArrayList("id")),
+ new StringLiteral("xxx"));
+ se = new ShowExportCommand("", "test", where6, null, -1);
+ ShowExportCommand finalSe2 = se;
+ Assertions.assertThrows(AnalysisException.class, () ->
finalSe2.validate(connectContext));
+
+ Expression where7 = new EqualTo(new
UnboundSlot(Lists.newArrayList("id")),
+ new StringLiteral("xxx"));
+ se = new ShowExportCommand("", "test", where7, null, -1);
+ ShowExportCommand finalSe3 = se;
+ Assertions.assertThrows(AnalysisException.class, () ->
finalSe3.validate(connectContext));
+
+ Expression equalTo1 = new EqualTo(new
UnboundSlot(Lists.newArrayList("STATE")),
+ new StringLiteral("PENDING"));
+ Expression equalTo2 = new EqualTo(new
UnboundSlot(Lists.newArrayList("LABEL")),
+ new StringLiteral("xxx"));
+ Expression where8 = new And(equalTo1, equalTo2);
+ se = new ShowExportCommand("", "test", where8, null, -1);
+ ShowExportCommand finalSe4 = se;
+ Assertions.assertThrows(AnalysisException.class, () ->
finalSe4.validate(connectContext));
+
+ Expression where9 = new EqualTo(new
UnboundSlot(Lists.newArrayList("state")),
+ new StringLiteral("FINISHED"));
+ se = new ShowExportCommand("", "", where9, null, -1);
+ ShowExportCommand finalSe5 = se;
+ Assertions.assertThrows(AnalysisException.class, () ->
finalSe5.validate(connectContext));
+
+ Expression where10 = new EqualTo(new
UnboundSlot(Lists.newArrayList("state")),
+ new StringLiteral("xxx"));
+ se = new ShowExportCommand("", "test", where10, null, -1);
+ ShowExportCommand finalSe6 = se;
+ Assertions.assertThrows(AnalysisException.class, () ->
finalSe6.validate(connectContext));
+ }
+}
diff --git
a/regression-test/suites/nereids_p0/show/test_nereids_show_export.groovy
b/regression-test/suites/nereids_p0/show/test_nereids_show_export.groovy
new file mode 100644
index 00000000000..25eb6bbd3e5
--- /dev/null
+++ b/regression-test/suites/nereids_p0/show/test_nereids_show_export.groovy
@@ -0,0 +1,166 @@
+// 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_export") {
+ sql """ drop database if exists test_nereids_show_export_db"""
+ sql """create database test_nereids_show_export_db"""
+ sql """use test_nereids_show_export_db;"""
+ sql """ DROP TABLE IF EXISTS test_nereids_show_export """
+
+ sql """
+ CREATE TABLE IF NOT EXISTS test_nereids_show_export (
+ `k1` bigint(20) NULL,
+ `k2` bigint(20) NULL
+ ) ENGINE=OLAP
+ COMMENT 'OLAP'
+ DISTRIBUTED BY HASH(`k1`) BUCKETS 2
+ PROPERTIES ("replication_allocation" = "tag.location.default: 1");
+ """
+ sql """ insert into test_nereids_show_export values (1,1),(2,2); """
+
+ // export config
+ def table_export_name = "test_nereids_show_export"
+ def outfile_path_prefix = """/tmp/test_export_basic"""
+ def uuid = UUID.randomUUID().toString()
+ def outFilePath = """${outfile_path_prefix}/${table_export_name}_${uuid}"""
+ def label = "label_xxyyzz_1"
+ def label1 = "label_xxyyzz_2"
+ def label2 = "label_xxyyzz_3"
+ def machine_user_name = "root"
+
+ def check_path_exists = { dir_path ->
+ mkdirRemotePathOnAllBE(machine_user_name, dir_path)
+ }
+
+ def delete_files = { dir_path ->
+ deleteRemotePathOnAllBE(machine_user_name, dir_path)
+ }
+
+ def waiting_export = { the_db, export_label ->
+ while (true) {
+ def res = sql """ show export from ${the_db} where label =
"${export_label}" """
+ logger.info("export state: " + res[0][2])
+ if (res[0][2] == "FINISHED") {
+ break;
+ } else if (res[0][2] == "CANCELLED") {
+ throw new IllegalStateException("""export failed:
${res[0][10]}""")
+ } else {
+ sleep(5000)
+ }
+ }
+ }
+
+ try {
+ // check export path
+ check_path_exists.call("${outFilePath}")
+
+ // exec export 1
+ sql """
+ EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/"
+ PROPERTIES(
+ "label" = "${label}",
+ "format" = "csv",
+ "column_separator"=",",
+ "data_consistency" = "none"
+ );
+ """
+ waiting_export.call("test_nereids_show_export_db", label)
+
+ // exec export 2
+ sql """
+ EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/"
+ PROPERTIES(
+ "label" = "${label1}",
+ "format" = "csv",
+ "column_separator"=",",
+ "data_consistency" = "none"
+ );
+ """
+ waiting_export.call("test_nereids_show_export_db", label1)
+
+ // exec export 3
+ sql """
+ EXPORT TABLE ${table_export_name} TO "file://${outFilePath}/"
+ PROPERTIES(
+ "label" = "${label2}",
+ "format" = "csv",
+ "column_separator"=",",
+ "data_consistency" = "none"
+ );
+ """
+ waiting_export.call("test_nereids_show_export_db", label2)
+
+ // test show
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
order by JobId")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
order by JobId limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
order by Label")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
order by Label limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where id = 123")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where id = 123 limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where id = 123 order by JobId")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where id = 123 order by JobId limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where id = 123 order by Label")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where id = 123 order by Label limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label like 'F%'")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label like 'F%' limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label like 'F%' order by JobId")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label like 'F%' order by JobId limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label like 'F%' order by Label")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label like 'F%' order by Label limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where state = 'FINISHED'")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where state = 'FINISHED' limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where state = 'FINISHED' order by JobId")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where state = 'FINISHED' order by JobId limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where state = 'FINISHED' order by state")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where state = 'FINISHED' order by state limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label = 'xxx'")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label = 'xxx' limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label = 'xxx' order by JobId")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label = 'xxx' order by JobId limit 1")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label = 'xxx' order by CreateTime")
+ checkNereidsExecute("SHOW EXPORT FROM test_nereids_show_export_db
where label = 'xxx' order by CreateTime limit 1")
+
+ def res1 = sql """SHOW EXPORT FROM test_nereids_show_export_db"""
+ assertEquals(3, res1.size())
+ def res2 = sql """SHOW EXPORT FROM test_nereids_show_export_db order
by JobId"""
+ assertEquals(3, res2.size())
+ def res3 = sql """SHOW EXPORT FROM test_nereids_show_export_db order
by JobId limit 1"""
+ assertEquals(1, res3.size())
+ def res4 = sql """SHOW EXPORT FROM test_nereids_show_export_db where
label = 'label_xxyyzz_2'"""
+ assertEquals(1, res4.size())
+ assertEquals("label_xxyyzz_2", res4.get(0).get(1))
+ def res5 = sql """SHOW EXPORT FROM test_nereids_show_export_db where
state = 'FINISHED'"""
+ assertEquals(3, res5.size())
+ def res6 = sql """SHOW EXPORT FROM test_nereids_show_export_db where
state = 'FINISHED' order by label"""
+ assertEquals(3, res6.size())
+ assertEquals("label_xxyyzz_1", res6.get(0).get(1))
+ def res7 = sql """SHOW EXPORT FROM test_nereids_show_export_db where
state = 'FINISHED' order by label desc"""
+ assertEquals(3, res7.size())
+ assertEquals("label_xxyyzz_3", res7.get(0).get(1))
+ def res8 = sql """SHOW EXPORT FROM test_nereids_show_export_db where
LABEL like 'label_xxyyzz_%'"""
+ assertEquals(3, res8.size())
+ def res9 = sql """SHOW EXPORT FROM test_nereids_show_export_db where
LABEL like 'label_xxyyzz_%' order by LABEL limit 1"""
+ assertEquals(1, res9.size())
+ assertEquals("label_xxyyzz_1", res9.get(0).get(1))
+ } finally {
+ try_sql("DROP TABLE IF EXISTS test_nereids_show_export")
+ try_sql("DROP DATABASE IF EXISTS test_nereids_show_export_db")
+ delete_files.call("${outFilePath}")
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]