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 4096dfad945 [Enhancement](nereids)support show build index (#51570)
4096dfad945 is described below
commit 4096dfad945b145734220d9974a70810ce290f5c
Author: lsy3993 <[email protected]>
AuthorDate: Tue Jun 10 10:14:21 2025 +0800
[Enhancement](nereids)support show build index (#51570)
---
.../antlr4/org/apache/doris/nereids/DorisParser.g4 | 4 +-
.../doris/common/proc/BuildIndexProcDir.java | 145 +++++++++++++
.../doris/nereids/parser/LogicalPlanBuilder.java | 35 +++
.../apache/doris/nereids/trees/plans/PlanType.java | 1 +
.../plans/commands/ShowBuildIndexCommand.java | 241 +++++++++++++++++++++
.../trees/plans/visitor/CommandVisitor.java | 5 +
.../plans/commands/ShowBuildIndexCommandTest.java | 110 ++++++++++
.../show/test_nereids_show_build_index.groovy | 102 +++++++++
8 files changed, 641 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 c6a2ca1eb79..12d63a4ed50 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
@@ -316,6 +316,8 @@ supportedShowStatement
| SHOW CREATE (DATABASE | SCHEMA) name=multipartIdentifier
#showCreateDatabase
| SHOW BACKUP ((FROM | IN) database=identifier)? wildWhere?
#showBackup
| SHOW BROKER
#showBroker
+ | SHOW BUILD INDEX ((FROM | IN) database=identifier)?
+ wildWhere? sortClause? limitClause?
#showBuildIndex
| SHOW DYNAMIC PARTITION TABLES ((FROM | IN)
database=multipartIdentifier)? #showDynamicPartition
| SHOW EVENTS ((FROM | IN) database=multipartIdentifier)? wildWhere?
#showEvents
| SHOW EXPORT ((FROM | IN) database=multipartIdentifier)? wildWhere?
@@ -483,8 +485,6 @@ unsupportedShowStatement
(FROM |IN) tableName=multipartIdentifier
((FROM | IN) database=multipartIdentifier)?
#showIndex
| SHOW CACHE HOTSPOT tablePath=STRING_LITERAL
#showCacheHotSpot
- | SHOW BUILD INDEX ((FROM | IN) database=multipartIdentifier)?
- wildWhere? sortClause? limitClause?
#showBuildIndex
;
createRoutineLoad
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BuildIndexProcDir.java
b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BuildIndexProcDir.java
index 9c381fe5676..6bf48d00260 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BuildIndexProcDir.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BuildIndexProcDir.java
@@ -28,6 +28,21 @@ import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.util.ListComparator;
import org.apache.doris.common.util.OrderByPair;
+import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
+import org.apache.doris.nereids.trees.expressions.EqualTo;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.GreaterThan;
+import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
+import org.apache.doris.nereids.trees.expressions.LessThan;
+import org.apache.doris.nereids.trees.expressions.LessThanEqual;
+import org.apache.doris.nereids.trees.expressions.Not;
+import org.apache.doris.nereids.trees.expressions.NullSafeEqual;
+import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral;
+import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal;
+import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal;
+import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+import org.apache.doris.nereids.types.coercion.DateLikeType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
@@ -110,6 +125,136 @@ public class BuildIndexProcDir implements
ProcDirInterface {
return true;
}
+ boolean filterResultExpression(String columnName, Comparable element,
HashMap<String, Expression> filter)
+ throws AnalysisException {
+ if (filter == null) {
+ return true;
+ }
+ Expression subExpr = filter.get(columnName.toLowerCase());
+ if (subExpr == null) {
+ return true;
+ }
+
+ if (subExpr instanceof ComparisonPredicate) {
+ return filterSubExpression(subExpr, element);
+ } else if (subExpr instanceof Not) {
+ subExpr = subExpr.child(0);
+ if (subExpr instanceof EqualTo) {
+ return !filterSubExpression(subExpr, element);
+ }
+ }
+
+ return false;
+ }
+
+ boolean filterSubExpression(Expression expr, Comparable element) throws
AnalysisException {
+ if (expr instanceof EqualTo && expr.child(1) instanceof
StringLikeLiteral) {
+ return ((StringLikeLiteral)
expr.child(1)).getValue().equals(element);
+ }
+
+ long leftVal;
+ long rightVal;
+ if (expr.child(1) instanceof
org.apache.doris.nereids.trees.expressions.literal.DateLiteral) {
+ DateLikeType dateLikeType;
+ if (expr.child(1) instanceof DateV2Literal) {
+ leftVal = (new
org.apache.doris.nereids.trees.expressions.literal.DateV2Literal(
+ (String) element)).getValue();
+ } else if (expr.child(1) instanceof DateTimeLiteral) {
+ leftVal = (new
org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral(
+ (String) element)).getValue();
+ } else if (expr.child(1) instanceof DateTimeV2Literal) {
+ dateLikeType = DateTimeV2Type.MAX;
+ leftVal = (new
org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal(
+ (DateTimeV2Type) dateLikeType, (String)
element)).getValue();
+ } else {
+ throw new AnalysisException("Invalid date type: " +
expr.child(1).getDataType());
+ }
+ rightVal =
((org.apache.doris.nereids.trees.expressions.literal.DateLiteral)
expr.child(1)).getValue();
+ if (expr instanceof EqualTo) {
+ return leftVal == rightVal;
+ } else if (expr instanceof NullSafeEqual) {
+ return leftVal == rightVal;
+ } else if (expr instanceof GreaterThan) {
+ return leftVal > rightVal;
+ } else if (expr instanceof GreaterThanEqual) {
+ return leftVal >= rightVal;
+ } else if (expr instanceof LessThan) {
+ return leftVal < rightVal;
+ } else if (expr instanceof LessThanEqual) {
+ return leftVal <= rightVal;
+ } else {
+ Preconditions.checkState(false, "No defined binary operator.");
+ }
+ }
+ return true;
+ }
+
+ public ProcResult fetchResultByFilterExpression(HashMap<String,
Expression> filter,
+ ArrayList<OrderByPair>
orderByPairs,
+ LimitElement
limitElement) throws AnalysisException {
+ Preconditions.checkNotNull(db);
+ Preconditions.checkNotNull(schemaChangeHandler);
+
+ List<List<Comparable>> indexChangeJobInfos =
schemaChangeHandler.getAllIndexChangeJobInfos(db);
+
+ //where
+ List<List<Comparable>> jobInfos;
+ if (filter == null || filter.size() == 0) {
+ jobInfos = indexChangeJobInfos;
+ } else {
+ jobInfos = Lists.newArrayList();
+ for (List<Comparable> infoStr : indexChangeJobInfos) {
+ if (infoStr.size() != TITLE_NAMES.size()) {
+ LOG.warn("indexChangeJobInfos.size() " +
indexChangeJobInfos.size()
+ + " not equal TITLE_NAMES.size() " +
TITLE_NAMES.size());
+ continue;
+ }
+ boolean isNeed = true;
+ for (int i = 0; i < infoStr.size(); i++) {
+ isNeed = filterResultExpression(TITLE_NAMES.get(i),
infoStr.get(i), filter);
+ if (!isNeed) {
+ break;
+ }
+ }
+ if (isNeed) {
+ jobInfos.add(infoStr);
+ }
+ }
+ }
+
+ // order by
+ if (orderByPairs != null) {
+ ListComparator<List<Comparable>> comparator = null;
+ OrderByPair[] orderByPairArr = new
OrderByPair[orderByPairs.size()];
+ comparator = new
ListComparator<List<Comparable>>(orderByPairs.toArray(orderByPairArr));
+ Collections.sort(jobInfos, comparator);
+ }
+
+ //limit
+ if (limitElement != null && limitElement.hasLimit()) {
+ int beginIndex = (int) limitElement.getOffset();
+ int endIndex = (int) (beginIndex + limitElement.getLimit());
+ if (endIndex > jobInfos.size()) {
+ endIndex = jobInfos.size();
+ }
+ if (beginIndex > endIndex) {
+ beginIndex = endIndex;
+ }
+ jobInfos = jobInfos.subList(beginIndex, endIndex);
+ }
+
+ BaseProcResult result = new BaseProcResult();
+ result.setNames(TITLE_NAMES);
+ for (List<Comparable> jobInfo : jobInfos) {
+ List<String> oneResult = new ArrayList<String>(jobInfos.size());
+ for (Comparable column : jobInfo) {
+ oneResult.add(column.toString());
+ }
+ result.addRow(oneResult);
+ }
+ return result;
+ }
+
public ProcResult fetchResultByFilter(HashMap<String, Expr> filter,
ArrayList<OrderByPair> orderByPairs,
LimitElement limitElement) throws
AnalysisException {
Preconditions.checkNotNull(db);
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 590fcb8ba65..6e4b1c57902 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
@@ -326,6 +326,7 @@ 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.ShowBuildIndexContext;
import org.apache.doris.nereids.DorisParser.ShowCatalogRecycleBinContext;
import org.apache.doris.nereids.DorisParser.ShowCharsetContext;
import org.apache.doris.nereids.DorisParser.ShowClustersContext;
@@ -692,6 +693,7 @@ 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.ShowBuildIndexCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCatalogCommand;
import
org.apache.doris.nereids.trees.plans.commands.ShowCatalogRecycleBinCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCharsetCommand;
@@ -6044,6 +6046,39 @@ public class LogicalPlanBuilder extends
DorisParserBaseVisitor<Object> {
return new ShowBrokerCommand();
}
+ @Override
+ public LogicalPlan visitShowBuildIndex(ShowBuildIndexContext ctx) {
+ String dbName = null;
+ Expression wildWhere = null;
+ List<OrderKey> orderKeys = null;
+ long limit = -1L;
+ long offset = 0L;
+ if (ctx.database != null) {
+ dbName = ctx.database.getText();
+ }
+ if (ctx.wildWhere() != null) {
+ wildWhere = getWildWhere(ctx.wildWhere());
+ }
+ if (ctx.sortClause() != null) {
+ orderKeys = visit(ctx.sortClause().sortItem(), OrderKey.class);
+ }
+ 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());
+ }
+ offset = ctx.limitClause().offset != null
+ ? Long.parseLong(ctx.limitClause().offset.getText())
+ : 0;
+ if (offset < 0) {
+ throw new ParseException("Offset requires non-negative
number", ctx.limitClause());
+ }
+ }
+ return new ShowBuildIndexCommand(dbName, wildWhere, orderKeys, limit,
offset);
+ }
+
@Override
public LogicalPlan visitDropRole(DropRoleContext ctx) {
String roleName = stripQuotes(ctx.name.getText());
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 04091e6c815..45df7fabbef 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
@@ -245,6 +245,7 @@ public enum PlanType {
SHOW_BACKUP_COMMAND,
SHOW_BLOCK_RULE_COMMAND,
SHOW_BROKER_COMMAND,
+ SHOW_BUILD_INDEX_COMMAND,
SHOW_CATALOG_COMMAND,
SHOW_CATALOG_RECYCLE_BIN_COMMAND,
SHOW_CHARSET_COMMAND,
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBuildIndexCommand.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBuildIndexCommand.java
new file mode 100644
index 00000000000..81341b6046c
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowBuildIndexCommand.java
@@ -0,0 +1,241 @@
+// 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.LimitElement;
+import org.apache.doris.analysis.RedirectStatus;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.DatabaseIf;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.UserException;
+import org.apache.doris.common.proc.BuildIndexProcDir;
+import org.apache.doris.common.proc.ProcNodeInterface;
+import org.apache.doris.common.proc.ProcService;
+import org.apache.doris.common.util.OrderByPair;
+import org.apache.doris.nereids.analyzer.UnboundSlot;
+import org.apache.doris.nereids.properties.OrderKey;
+import org.apache.doris.nereids.trees.expressions.And;
+import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
+import org.apache.doris.nereids.trees.expressions.CompoundPredicate;
+import org.apache.doris.nereids.trees.expressions.EqualTo;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.Not;
+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.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateTimeV2Type;
+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.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * show build index command
+ */
+public class ShowBuildIndexCommand extends ShowCommand {
+ public static final ImmutableList<String> TITLE_NAMES = new
ImmutableList.Builder<String>()
+ .add("JobId").add("TableName")
+ .add("PartitionName").add("AlterInvertedIndexes")
+ .add("CreateTime").add("FinishTime")
+ .add("TransactionId").add("State")
+ .add("Msg").add("Progress")
+ .build();
+ private static String TABLE_NAME = "tablename";
+ private static String PARTITION_NAME = "partitionname";
+ private static String STATE = "state";
+ private static String CREATE_TIME = "createtime";
+ private static String FINISH_TIME = "finishtime";
+ private static final Logger LOG =
LogManager.getLogger(ShowBuildIndexCommand.class);
+ private String dbName;
+ private final Expression wildWhere;
+ private final long limit;
+ private final long offset;
+ private final List<OrderKey> orderKeys;
+ private HashMap<String, Expression> filterMap;
+ private ArrayList<OrderByPair> orderByPairs;
+ private ProcNodeInterface node;
+
+ /**
+ * constructor for show build index
+ */
+ public ShowBuildIndexCommand(String dbName, Expression wildWhere,
+ List<OrderKey> orderKeys, long limit,
long offset) {
+ super(PlanType.SHOW_BUILD_INDEX_COMMAND);
+ this.dbName = dbName;
+ this.wildWhere = wildWhere;
+ this.orderKeys = orderKeys;
+ this.limit = limit;
+ this.offset = offset;
+ this.filterMap = new HashMap<String, Expression>();
+ }
+
+ private void validate(ConnectContext ctx) throws UserException {
+ if (Strings.isNullOrEmpty(dbName)) {
+ dbName = ctx.getDatabase();
+ if (Strings.isNullOrEmpty(dbName)) {
+ ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
+ }
+ }
+
+ analyzeSubPredicate(wildWhere);
+
+ // then process the order by
+ orderByPairs = getOrderByPairs(orderKeys, TITLE_NAMES);
+ }
+
+ private void analyzeSubPredicate(Expression subExpr) throws
AnalysisException {
+ if (subExpr == null) {
+ return;
+ }
+ if (subExpr instanceof CompoundPredicate) {
+ if (!(subExpr instanceof And)) {
+ throw new AnalysisException("Only allow compound predicate
with operator AND");
+ }
+ for (Expression child : subExpr.children()) {
+ analyzeSubPredicate(child);
+ }
+ return;
+ }
+
+ boolean isNotExpr = false;
+ if (subExpr instanceof Not) {
+ isNotExpr = true;
+ subExpr = subExpr.child(0);
+ if (!(subExpr instanceof EqualTo)) {
+ throw new AnalysisException("Only operator =|>=|<=|>|<|!=|like
are supported.");
+ }
+ }
+
+ getPredicateValue(subExpr, isNotExpr);
+ }
+
+ private void getPredicateValue(Expression subExpr, boolean isNotExpr)
throws AnalysisException {
+ if (!(subExpr instanceof ComparisonPredicate)) {
+ throw new AnalysisException("The operator =|>=|<=|>|<|!= are
supported.");
+ }
+
+ ComparisonPredicate binaryPredicate = (ComparisonPredicate) subExpr;
+ if (!(subExpr.child(0) instanceof UnboundSlot)) {
+ throw new AnalysisException("Only support column = xxx syntax.");
+ }
+ String leftKey = ((UnboundSlot)
subExpr.child(0)).getName().toLowerCase();
+ if (leftKey.equalsIgnoreCase(TABLE_NAME)
+ || leftKey.equalsIgnoreCase(STATE)
+ || leftKey.equalsIgnoreCase(PARTITION_NAME)) {
+ if (!(subExpr.child(1) instanceof StringLikeLiteral) ||
!(binaryPredicate instanceof EqualTo)) {
+ throw new AnalysisException("Where clause : TableName =
\"table1\" or "
+ + "State =
\"FINISHED|CANCELLED|RUNNING|PENDING|WAITING_TXN\"");
+ }
+ } else if (leftKey.equalsIgnoreCase(CREATE_TIME) ||
leftKey.equalsIgnoreCase(FINISH_TIME)) {
+ if (!(subExpr.child(1) instanceof StringLikeLiteral)) {
+ throw new AnalysisException("Where clause :
CreateTime/FinishTime =|>=|<=|>|<|!= "
+ + "\"2019-12-02|2019-12-02 14:54:00\"");
+ }
+ Expression left = subExpr.child(0);
+ Expression right =
subExpr.child(1).castTo(Config.enable_date_conversion
+ ? DateTimeV2Type.MAX : DateTimeType.INSTANCE);
+ subExpr = subExpr.withChildren(left, right);
+ } else {
+ throw new AnalysisException(
+ "The columns of
TableName/PartitionName/CreateTime/FinishTime/State are supported.");
+ }
+ filterMap.put(leftKey.toLowerCase(), isNotExpr ? new Not(subExpr) :
subExpr);
+ }
+
+ private void analyze(ConnectContext ctx) throws UserException {
+ DatabaseIf db =
ctx.getCurrentCatalog().getDbOrAnalysisException(dbName);
+ // build proc path
+ StringBuilder sb = new StringBuilder();
+ sb.append("/jobs/");
+ sb.append(db.getId());
+ sb.append("/build_index");
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("process SHOW PROC '{}';", sb.toString());
+ }
+ // create show proc stmt
+ // '/jobs/db_name/build_index/
+ node = ProcService.getInstance().open(sb.toString());
+ if (node == null) {
+ throw new AnalysisException("Failed to show build index");
+ }
+ }
+
+ @VisibleForTesting
+ protected ShowResultSet handleShowBuildIndex(ConnectContext ctx,
StmtExecutor executor) throws Exception {
+ // first validate the where
+ validate(ctx);
+
+ // then analyze
+ analyze(ctx);
+
+ Preconditions.checkNotNull(node);
+ LimitElement limitElement = null;
+ if (limit > 0) {
+ limitElement = new LimitElement(offset == -1L ? 0 : offset, limit);
+ }
+
+ List<List<String>> rows = ((BuildIndexProcDir)
node).fetchResultByFilterExpression(
+ filterMap, orderByPairs, limitElement).getRows();
+ return new ShowResultSet(getMetaData(), rows);
+ }
+
+ @Override
+ public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor)
throws Exception {
+ return handleShowBuildIndex(ctx, executor);
+ }
+
+ @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();
+ }
+
+ @Override
+ public RedirectStatus toRedirectStatus() {
+ if (ConnectContext.get().getSessionVariable().getForwardToMaster()) {
+ return RedirectStatus.FORWARD_NO_SYNC;
+ } else {
+ return RedirectStatus.NO_FORWARD;
+ }
+ }
+
+ @Override
+ public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+ return visitor.visitShowBuildIndexCommand(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 ba19d4f4ebd..95376c75070 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.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.ShowBuildIndexCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCatalogCommand;
import
org.apache.doris.nereids.trees.plans.commands.ShowCatalogRecycleBinCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCharsetCommand;
@@ -831,6 +832,10 @@ public interface CommandVisitor<R, C> {
return visitCommand(showBrokerCommand, context);
}
+ default R visitShowBuildIndexCommand(ShowBuildIndexCommand
showBuildIndexCommand, C context) {
+ return visitCommand(showBuildIndexCommand, context);
+ }
+
default R visitShowLoadCommand(ShowLoadCommand showLoadCommand, C context)
{
return visitCommand(showLoadCommand, context);
}
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowBuildIndexCommandTest.java
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowBuildIndexCommandTest.java
new file mode 100644
index 00000000000..98331748340
--- /dev/null
+++
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowBuildIndexCommandTest.java
@@ -0,0 +1,110 @@
+// 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.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.GreaterThan;
+import org.apache.doris.nereids.trees.expressions.GreaterThanEqual;
+import org.apache.doris.nereids.trees.expressions.LessThan;
+import org.apache.doris.nereids.trees.expressions.LessThanEqual;
+import org.apache.doris.nereids.trees.expressions.Not;
+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.Test;
+
+import java.util.List;
+
+public class ShowBuildIndexCommandTest extends TestWithFeService {
+ @Override
+ protected void runBeforeAll() throws Exception {
+ createDatabase("test");
+ }
+
+ @Test
+ void testHandleShowBuildIndex() throws Exception {
+ // test where is null but db is not null
+ ShowBuildIndexCommand sa = new ShowBuildIndexCommand("test", null,
null, -1, -1);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ // different limit and offset
+ sa = new ShowBuildIndexCommand("test", null, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+ sa = new ShowBuildIndexCommand("test", null, null, 2, 1);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ // order by
+ UnboundSlot key = new UnboundSlot(Lists.newArrayList("JobId"));
+ List<OrderKey> orderKeys = Lists.newArrayList(new OrderKey(key, true,
true));
+ sa = new ShowBuildIndexCommand("test", null, orderKeys, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where1 = new EqualTo(new
UnboundSlot(Lists.newArrayList("TableName")),
+ new StringLiteral("xxx"));
+ sa = new ShowBuildIndexCommand("test", where1, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where2 = new Not(where1);
+ sa = new ShowBuildIndexCommand("test", where2, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where3 = new EqualTo(new
UnboundSlot(Lists.newArrayList("State")),
+ new StringLiteral("FINISHED"));
+ sa = new ShowBuildIndexCommand("test", where3, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where4 = new EqualTo(new
UnboundSlot(Lists.newArrayList("CreateTime")),
+ new StringLiteral("2025-06-04 21:53:53"));
+ sa = new ShowBuildIndexCommand("test", where4, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where5 = new EqualTo(new
UnboundSlot(Lists.newArrayList("FinishTime")),
+ new StringLiteral("2025-06-04 21:53:54"));
+ sa = new ShowBuildIndexCommand("test", where5, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where6 = new LessThan(new
UnboundSlot(Lists.newArrayList("FinishTime")),
+ new StringLiteral("2025-06-04 21:53:54"));
+ sa = new ShowBuildIndexCommand("test", where6, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where7 = new LessThanEqual(new
UnboundSlot(Lists.newArrayList("FinishTime")),
+ new StringLiteral("2025-06-04 21:53:54"));
+ sa = new ShowBuildIndexCommand("test", where7, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where8 = new GreaterThanEqual(new
UnboundSlot(Lists.newArrayList("FinishTime")),
+ new StringLiteral("2025-06-04 21:53:54"));
+ sa = new ShowBuildIndexCommand("test", where8, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where9 = new GreaterThan(new
UnboundSlot(Lists.newArrayList("FinishTime")),
+ new StringLiteral("2025-06-04 21:53:54"));
+ sa = new ShowBuildIndexCommand("test", where9, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+
+ Expression where10 = new EqualTo(new
UnboundSlot(Lists.newArrayList("PartitionName")),
+ new StringLiteral("xxx"));
+ sa = new ShowBuildIndexCommand("test", where10, null, 1, 0);
+ sa.handleShowBuildIndex(connectContext, null);
+ }
+}
diff --git
a/regression-test/suites/nereids_p0/show/test_nereids_show_build_index.groovy
b/regression-test/suites/nereids_p0/show/test_nereids_show_build_index.groovy
new file mode 100644
index 00000000000..73e87efb103
--- /dev/null
+++
b/regression-test/suites/nereids_p0/show/test_nereids_show_build_index.groovy
@@ -0,0 +1,102 @@
+// 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_build_index") {
+ if (!isCloudMode()) {
+ sql "DROP DATABASE IF EXISTS test_show_build_index"
+ sql "CREATE DATABASE IF NOT EXISTS test_show_build_index"
+ sql "DROP TABLE IF EXISTS
test_show_build_index.test_show_build_index_tbl1"
+ sql "DROP TABLE IF EXISTS
test_show_build_index.test_show_build_index_tbl2"
+ sql """
+ CREATE TABLE IF NOT EXISTS
test_show_build_index.test_show_build_index_tbl1 (
+ `user_id` LARGEINT NOT NULL COMMENT "用户id",
+ `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+ `city` VARCHAR(20) COMMENT "用户所在城市",
+ `age` SMALLINT COMMENT "用户年龄",
+ `sex` TINYINT COMMENT "用户性别",
+ `note` TEXT COMMENT "备注",
+ INDEX idx_user_id1 (`user_id`) USING INVERTED COMMENT '',
+ INDEX idx_note1 (`note`) USING INVERTED PROPERTIES("parser" =
"english") COMMENT ''
+ )
+ DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY
HASH(`user_id`)
+ PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format"
= "V1" );
+ """
+ sql """
+ CREATE TABLE IF NOT EXISTS
test_show_build_index.test_show_build_index_tbl2 (
+ `user_id` LARGEINT NOT NULL COMMENT "用户id",
+ `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+ `city` VARCHAR(20) COMMENT "用户所在城市",
+ `age` SMALLINT COMMENT "用户年龄",
+ `sex` TINYINT COMMENT "用户性别",
+ `note` TEXT COMMENT "备注",
+ INDEX idx_user_id2 (`user_id`) USING INVERTED COMMENT '',
+ INDEX idx_note2 (`note`) USING INVERTED PROPERTIES("parser" =
"english") COMMENT ''
+ )
+ DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY
HASH(`user_id`)
+ PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format"
= "V1" );
+ """
+ sql "BUILD INDEX idx_user_id1 ON
test_show_build_index.test_show_build_index_tbl1;"
+ sql "BUILD INDEX idx_user_id2 ON
test_show_build_index.test_show_build_index_tbl2;"
+ sleep(30000)
+ checkNereidsExecute("show build index from test_show_build_index;")
+ checkNereidsExecute("show build index in test_show_build_index;")
+ checkNereidsExecute("show build index from test_show_build_index order
by JobId;")
+ checkNereidsExecute("show build index from test_show_build_index order
by JobId desc;")
+ checkNereidsExecute("show build index from test_show_build_index where
TableName = 'test_show_build_index_tbl2';")
+ checkNereidsExecute("show build index from test_show_build_index where
PartitionName = 'test_show_build_index_tbl1';")
+ checkNereidsExecute("show build index from test_show_build_index where
CreateTime = '2025-06-04 20:58:27';")
+ checkNereidsExecute("show build index from test_show_build_index where
FinishTime = '2025-06-04 20:58:27';")
+ checkNereidsExecute("show build index from test_show_build_index where
State = 'FINISHED';")
+ checkNereidsExecute("show build index from test_show_build_index where
State = 'FINISHED' order by JobId;")
+ checkNereidsExecute("show build index from test_show_build_index where
State = 'FINISHED' order by TableName;")
+ checkNereidsExecute("show build index from test_show_build_index where
State = 'FINISHED' order by TableName limit 1;")
+ checkNereidsExecute("show build index from test_show_build_index where
State = 'FINISHED' order by TableName limit 1,1;")
+ checkNereidsExecute("show build index from test_show_build_index where
State = 'FINISHED' and CreateTime = '2025-06-04 21:01:50';")
+ checkNereidsExecute("show build index from test_show_build_index where
FinishTime != '2025-06-04 21:53:48';")
+ checkNereidsExecute("show build index from test_show_build_index where
FinishTime >= '2025-06-04 21:53:48';")
+ checkNereidsExecute("show build index from test_show_build_index where
FinishTime > '2025-06-04 21:53:48';")
+ checkNereidsExecute("show build index from test_show_build_index where
FinishTime <= '2025-06-04 21:53:48';")
+ checkNereidsExecute("show build index from test_show_build_index where
FinishTime < '2025-06-04 21:53:48';")
+ checkNereidsExecute("show build index from test_show_build_index where
TableName != 'test_show_build_index_tbl2';")
+ checkNereidsExecute("show build index from test_show_build_index where
CreateTime >= '2025-06-05 22:48:08';")
+ checkNereidsExecute("show build index from test_show_build_index where
CreateTime > '2025-06-05 22:48:08';")
+ checkNereidsExecute("show build index from test_show_build_index where
CreateTime <= '2025-06-05 22:48:08';")
+ checkNereidsExecute("show build index from test_show_build_index where
CreateTime < '2025-06-05 22:48:08';")
+
+ def res1 = sql """show build index from test_show_build_index"""
+ assertEquals(2, res1.size())
+ def res2 = sql """show build index from test_show_build_index order by
TableName"""
+ assertEquals(2, res2.size())
+ assertEquals("test_show_build_index_tbl1", res2.get(0).get(1))
+ def res3 = sql """show build index from test_show_build_index order by
TableName limit 1"""
+ assertEquals(1, res3.size())
+ assertEquals("test_show_build_index_tbl1", res3.get(0).get(1))
+ def res7 = sql """show build index from test_show_build_index where
TableName = 'test_show_build_index_tbl2';"""
+ assertEquals(1, res7.size())
+ assertEquals("test_show_build_index_tbl2", res7.get(0).get(2))
+
+ assertThrows(Exception.class, {
+ sql """show build index from test_show_build_index where JobId =
1749041691284;"""
+ })
+ assertThrows(Exception.class, {
+ sql """show build index from test_show_build_index where JobId =
1749041691284 or TableName = 'xx';"""
+ })
+ assertThrows(Exception.class, {
+ sql """show build index from test_show_build_index where indexname
= 'xx';"""
+ })
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]