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 <110876560+lsy3...@users.noreply.github.com>
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: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org


Reply via email to