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