This is an automated email from the ASF dual-hosted git repository.

englefly pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 9e9034930ce [Chore](nereids) Remove ShowTabletStmt (#51578)
9e9034930ce is described below

commit 9e9034930cece61b1b8a93c28ee9b55e1b5e40a2
Author: Jensen <[email protected]>
AuthorDate: Tue Jun 10 09:45:52 2025 +0800

    [Chore](nereids) Remove ShowTabletStmt (#51578)
    
    ### What problem does this PR solve?
    Remove ShowTabletStmt
---
 fe/fe-core/src/main/cup/sql_parser.cup             |   8 -
 .../org/apache/doris/analysis/ShowTabletStmt.java  | 333 ---------------------
 .../java/org/apache/doris/qe/ShowExecutor.java     | 194 ------------
 .../apache/doris/catalog/TempPartitionTest.java    |  17 +-
 .../apache/doris/catalog/TruncateTableTest.java    |  19 +-
 5 files changed, 24 insertions(+), 547 deletions(-)

diff --git a/fe/fe-core/src/main/cup/sql_parser.cup 
b/fe/fe-core/src/main/cup/sql_parser.cup
index c859aa206d4..73327b9a3b7 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -4503,18 +4503,10 @@ show_param ::=
     {:
         RESULT = new ShowPartitionIdStmt(partitionId);
     :}
-    | KW_TABLET INTEGER_LITERAL:tabletId
-    {:
-        RESULT = new ShowTabletStmt(null, tabletId);
-    :}
     | KW_TABLETS KW_BELONG integer_list:tabletIds
     {:
         RESULT = new ShowTabletsBelongStmt(tabletIds);
     :}
-    | KW_TABLETS KW_FROM table_name:dbTblName 
opt_partition_names:partitionNames opt_wild_where order_by_clause:orderByClause 
limit_clause:limitClause
-    {:
-        RESULT = new ShowTabletStmt(dbTblName, -1L, partitionNames, 
parser.where, orderByClause, limitClause);
-    :}
     | KW_PROPERTY opt_user:user opt_wild_where
     {:
         RESULT = new ShowUserPropertyStmt(user, parser.wild, false);
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java
deleted file mode 100644
index 646d6a80639..00000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletStmt.java
+++ /dev/null
@@ -1,333 +0,0 @@
-// 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.analysis;
-
-import org.apache.doris.catalog.Column;
-import org.apache.doris.catalog.Env;
-import org.apache.doris.catalog.Replica;
-import org.apache.doris.catalog.ScalarType;
-import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
-import org.apache.doris.common.UserException;
-import org.apache.doris.common.proc.TabletsProcDir;
-import org.apache.doris.common.util.OrderByPair;
-import org.apache.doris.common.util.Util;
-import org.apache.doris.mysql.privilege.PrivPredicate;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.ShowResultSetMetaData;
-
-import com.google.common.base.Strings;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class ShowTabletStmt extends ShowStmt implements NotFallbackInParser {
-    private TableName dbTableName;
-    private String dbName;
-    private String tableName;
-    private long tabletId;
-    private PartitionNames partitionNames;
-    private Expr whereClause;
-    private List<OrderByElement> orderByElements;
-    private LimitElement limitElement;
-
-    private long version;
-    private long backendId;
-    private String indexName;
-    private Replica.ReplicaState replicaState;
-    private ArrayList<OrderByPair> orderByPairs;
-
-    private boolean isShowSingleTablet;
-
-    public ShowTabletStmt(TableName dbTableName, long tabletId) {
-        this(dbTableName, tabletId, null, null, null, null);
-    }
-
-    public ShowTabletStmt(TableName dbTableName, long tabletId, PartitionNames 
partitionNames,
-            Expr whereClause, List<OrderByElement> orderByElements, 
LimitElement limitElement) {
-        this.dbTableName = dbTableName;
-        if (dbTableName == null) {
-            this.dbName = null;
-            this.tableName = null;
-            this.isShowSingleTablet = true;
-            this.indexName = null;
-        } else {
-            this.dbName = dbTableName.getDb();
-            this.tableName = dbTableName.getTbl();
-            this.isShowSingleTablet = false;
-            this.indexName = Strings.emptyToNull(indexName);
-        }
-        this.tabletId = tabletId;
-        this.partitionNames = partitionNames;
-        this.whereClause = whereClause;
-        this.orderByElements = orderByElements;
-        this.limitElement = limitElement;
-
-        this.version = -1;
-        this.backendId = -1;
-        this.indexName = null;
-        this.replicaState = null;
-        this.orderByPairs = null;
-    }
-
-    public String getDbName() {
-        return dbName;
-    }
-
-    public String getTableName() {
-        return tableName;
-    }
-
-    public long getTabletId() {
-        return tabletId;
-    }
-
-    public boolean isShowSingleTablet() {
-        return isShowSingleTablet;
-    }
-
-    public boolean hasOffset() {
-        return limitElement != null && limitElement.hasOffset();
-    }
-
-    public long getOffset() {
-        return limitElement.getOffset();
-    }
-
-    public boolean hasPartition() {
-        return partitionNames != null;
-    }
-
-    public PartitionNames getPartitionNames() {
-        return partitionNames;
-    }
-
-    public boolean hasLimit() {
-        return limitElement != null && limitElement.hasLimit();
-    }
-
-    public long getLimit() {
-        return limitElement.getLimit();
-    }
-
-    public long getVersion() {
-        return version;
-    }
-
-    public long getBackendId() {
-        return backendId;
-    }
-
-    public String getIndexName() {
-        return indexName;
-    }
-
-    public List<OrderByPair> getOrderByPairs() {
-        return orderByPairs;
-    }
-
-    public Replica.ReplicaState getReplicaState() {
-        return replicaState;
-    }
-
-    @Override
-    public void analyze(Analyzer analyzer) throws UserException {
-        // check access first
-        if 
(!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), 
PrivPredicate.ADMIN)) {
-            
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, 
"SHOW TABLET");
-        }
-
-        super.analyze(analyzer);
-        if (dbTableName != null) {
-            dbTableName.analyze(analyzer);
-            // disallow external catalog
-            Util.prohibitExternalCatalog(dbTableName.getCtl(), 
this.getClass().getSimpleName());
-        }
-        if (!isShowSingleTablet && Strings.isNullOrEmpty(dbName)) {
-            dbName = analyzer.getDefaultDb();
-            if (Strings.isNullOrEmpty(dbName)) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
-            }
-        }
-
-        if (partitionNames != null) {
-            partitionNames.analyze(analyzer);
-        }
-
-        if (limitElement != null) {
-            limitElement.analyze(analyzer);
-        }
-
-        // analyze where clause if not null
-        if (whereClause != null) {
-            if (whereClause instanceof CompoundPredicate) {
-                CompoundPredicate cp = (CompoundPredicate) whereClause;
-                if (cp.getOp() != 
org.apache.doris.analysis.CompoundPredicate.Operator.AND) {
-                    throw new AnalysisException("Only allow compound predicate 
with operator AND");
-                }
-
-                analyzeSubPredicate(cp.getChild(0));
-                analyzeSubPredicate(cp.getChild(1));
-            } else {
-                analyzeSubPredicate(whereClause);
-            }
-        }
-
-        // order by
-        if (orderByElements != null && !orderByElements.isEmpty()) {
-            orderByPairs = new ArrayList<OrderByPair>();
-            for (OrderByElement orderByElement : orderByElements) {
-                if (!(orderByElement.getExpr() instanceof SlotRef)) {
-                    throw new AnalysisException("Should order by column");
-                }
-                SlotRef slotRef = (SlotRef) orderByElement.getExpr();
-                int index = 
TabletsProcDir.analyzeColumn(slotRef.getColumnName());
-                OrderByPair orderByPair = new OrderByPair(index, 
!orderByElement.getIsAsc());
-                orderByPairs.add(orderByPair);
-            }
-        }
-    }
-
-    private void analyzeSubPredicate(Expr subExpr) throws AnalysisException {
-        if (subExpr == null) {
-            return;
-        }
-        if (subExpr instanceof CompoundPredicate) {
-            CompoundPredicate cp = (CompoundPredicate) subExpr;
-            if (cp.getOp() != 
org.apache.doris.analysis.CompoundPredicate.Operator.AND) {
-                throw new AnalysisException("Only allow compound predicate 
with operator AND");
-            }
-
-            analyzeSubPredicate(cp.getChild(0));
-            analyzeSubPredicate(cp.getChild(1));
-            return;
-        }
-        boolean valid = true;
-        do {
-            if (!(subExpr instanceof  BinaryPredicate)) {
-                valid = false;
-                break;
-            }
-            BinaryPredicate binaryPredicate = (BinaryPredicate) subExpr;
-            if (binaryPredicate.getOp() != BinaryPredicate.Operator.EQ) {
-                valid = false;
-                break;
-            }
-
-            if (!(subExpr.getChild(0) instanceof SlotRef)) {
-                valid = false;
-                break;
-            }
-            String leftKey = ((SlotRef) subExpr.getChild(0)).getColumnName();
-            if (leftKey.equalsIgnoreCase("version")) {
-                if (!(subExpr.getChild(1) instanceof IntLiteral) || version > 
-1) {
-                    valid = false;
-                    break;
-                }
-                version = ((IntLiteral) subExpr.getChild(1)).getValue();
-            } else if (leftKey.equalsIgnoreCase("backendid")) {
-                if (!(subExpr.getChild(1) instanceof IntLiteral) || backendId 
> -1) {
-                    valid = false;
-                    break;
-                }
-                backendId = ((IntLiteral) subExpr.getChild(1)).getValue();
-            } else if (leftKey.equalsIgnoreCase("indexname")) {
-                if (!(subExpr.getChild(1) instanceof StringLiteral) || 
indexName != null) {
-                    valid = false;
-                    break;
-                }
-                indexName = ((StringLiteral) subExpr.getChild(1)).getValue();
-            } else if (leftKey.equalsIgnoreCase("state")) {
-                if (!(subExpr.getChild(1) instanceof StringLiteral) || 
replicaState != null) {
-                    valid = false;
-                    break;
-                }
-                String state = ((StringLiteral) 
subExpr.getChild(1)).getValue().toUpperCase();
-                try {
-                    replicaState = Replica.ReplicaState.valueOf(state);
-                } catch (Exception e) {
-                    replicaState = null;
-                    valid = false;
-                    break;
-                }
-            } else {
-                valid = false;
-                break;
-            }
-        } while (false);
-
-        if (!valid) {
-            throw new AnalysisException("Where clause should looks like: 
Version = \"version\","
-                    + " or state = \"NORMAL|ROLLUP|CLONE|DECOMMISSION\", or 
BackendId = 10000,"
-                    + " indexname=\"rollup_name\" or compound predicate with 
operator AND");
-        }
-    }
-
-    @Override
-    public String toSql() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("SHOW TABLET ");
-        if (isShowSingleTablet) {
-            sb.append(tabletId);
-        } else {
-            sb.append(" FROM 
").append("`").append(dbName).append("`.`").append(tableName).append("`");
-        }
-        if (limitElement != null) {
-            if (limitElement.hasOffset() && limitElement.hasLimit()) {
-                sb.append(" 
").append(limitElement.getOffset()).append(",").append(limitElement.getLimit());
-            } else if (limitElement.hasLimit()) {
-                sb.append(" ").append(limitElement.getLimit());
-            }
-        }
-        return sb.toString();
-    }
-
-    @Override
-    public ShowResultSetMetaData getMetaData() {
-        ShowResultSetMetaData.Builder builder = 
ShowResultSetMetaData.builder();
-        if (isShowSingleTablet) {
-            builder.addColumn(new Column("DbName", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("TableName", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("PartitionName", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("IndexName", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("DbId", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("TableId", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("PartitionId", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("IndexId", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("IsSync", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("Order", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("QueryHits", 
ScalarType.createVarchar(30)));
-            builder.addColumn(new Column("DetailCmd", 
ScalarType.createVarchar(30)));
-        } else {
-            for (String title : TabletsProcDir.TITLE_NAMES) {
-                builder.addColumn(new Column(title, 
ScalarType.createVarchar(30)));
-            }
-        }
-        return builder.build();
-    }
-
-    @Override
-    public RedirectStatus getRedirectStatus() {
-        if (ConnectContext.get().getSessionVariable().getForwardToMaster()) {
-            return RedirectStatus.FORWARD_NO_SYNC;
-        } else {
-            return RedirectStatus.NO_FORWARD;
-        }
-    }
-}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java 
b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index 931a077b9f2..20e71a1f830 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -104,7 +104,6 @@ import org.apache.doris.analysis.ShowTableIdStmt;
 import org.apache.doris.analysis.ShowTableStatsStmt;
 import org.apache.doris.analysis.ShowTableStatusStmt;
 import org.apache.doris.analysis.ShowTableStmt;
-import org.apache.doris.analysis.ShowTabletStmt;
 import org.apache.doris.analysis.ShowTabletStorageFormatStmt;
 import org.apache.doris.analysis.ShowTabletsBelongStmt;
 import org.apache.doris.analysis.ShowTransactionStmt;
@@ -132,8 +131,6 @@ import org.apache.doris.catalog.Function;
 import org.apache.doris.catalog.FunctionUtil;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.MTMV;
-import org.apache.doris.catalog.MaterializedIndex;
-import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.MaterializedIndexMeta;
 import org.apache.doris.catalog.MetadataViewer;
 import org.apache.doris.catalog.OlapTable;
@@ -148,7 +145,6 @@ import org.apache.doris.catalog.StorageVault;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.TableIf;
 import org.apache.doris.catalog.TableIf.TableType;
-import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.catalog.TabletMeta;
 import org.apache.doris.catalog.View;
@@ -183,7 +179,6 @@ import org.apache.doris.common.proc.PartitionsProcDir;
 import org.apache.doris.common.proc.ProcNodeInterface;
 import org.apache.doris.common.proc.RollupProcDir;
 import org.apache.doris.common.proc.SchemaChangeProcDir;
-import org.apache.doris.common.proc.TabletsProcDir;
 import org.apache.doris.common.proc.TrashProcDir;
 import org.apache.doris.common.proc.TrashProcNode;
 import org.apache.doris.common.util.DebugUtil;
@@ -231,7 +226,6 @@ import 
org.apache.doris.statistics.PartitionColumnStatisticCacheKey;
 import org.apache.doris.statistics.ResultRow;
 import org.apache.doris.statistics.StatisticsRepository;
 import org.apache.doris.statistics.TableStatsMeta;
-import org.apache.doris.statistics.query.QueryStatsUtil;
 import org.apache.doris.statistics.util.StatisticsUtil;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.Diagnoser;
@@ -278,7 +272,6 @@ import java.time.ZoneId;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
@@ -387,8 +380,6 @@ public class ShowExecutor {
             handleShowPartitions();
         } else if (stmt instanceof ShowPartitionIdStmt) {
             handleShowPartitionId();
-        } else if (stmt instanceof ShowTabletStmt) {
-            handleShowTablet();
         } else if (stmt instanceof ShowBackupStmt) {
             handleShowBackup();
         } else if (stmt instanceof ShowRestoreStmt) {
@@ -2032,191 +2023,6 @@ public class ShowExecutor {
         resultSet = new ShowResultSet(showStmt.getMetaData(), rows);
     }
 
-    private void handleShowTablet() throws AnalysisException {
-        ShowTabletStmt showStmt = (ShowTabletStmt) stmt;
-        List<List<String>> rows = Lists.newArrayList();
-
-        Env env = Env.getCurrentEnv();
-        if (showStmt.isShowSingleTablet()) {
-            long tabletId = showStmt.getTabletId();
-            TabletInvertedIndex invertedIndex = Env.getCurrentInvertedIndex();
-            TabletMeta tabletMeta = invertedIndex.getTabletMeta(tabletId);
-            Long dbId = tabletMeta != null ? tabletMeta.getDbId() : 
TabletInvertedIndex.NOT_EXIST_VALUE;
-            String dbName = FeConstants.null_string;
-            Long tableId = tabletMeta != null ? tabletMeta.getTableId() : 
TabletInvertedIndex.NOT_EXIST_VALUE;
-            String tableName = FeConstants.null_string;
-            Long partitionId = tabletMeta != null ? 
tabletMeta.getPartitionId() : TabletInvertedIndex.NOT_EXIST_VALUE;
-            String partitionName = FeConstants.null_string;
-            Long indexId = tabletMeta != null ? tabletMeta.getIndexId() : 
TabletInvertedIndex.NOT_EXIST_VALUE;
-            String indexName = FeConstants.null_string;
-            Boolean isSync = true;
-            long queryHits = 0L;
-
-            int tabletIdx = -1;
-            // check real meta
-            do {
-                Database db = env.getInternalCatalog().getDbNullable(dbId);
-                if (db == null) {
-                    isSync = false;
-                    break;
-                }
-                dbName = db.getFullName();
-                Table table = db.getTableNullable(tableId);
-                if (!(table instanceof OlapTable)) {
-                    isSync = false;
-                    break;
-                }
-                if (Config.enable_query_hit_stats) {
-                    MaterializedIndex mi = ((OlapTable) 
table).getPartition(partitionId).getIndex(indexId);
-                    if (mi != null) {
-                        Tablet t = mi.getTablet(tabletId);
-                        for (Replica r : t.getReplicas()) {
-                            queryHits += 
QueryStatsUtil.getMergedReplicaStats(r.getId());
-                        }
-                    }
-                }
-
-                table.readLock();
-                try {
-                    tableName = table.getName();
-                    OlapTable olapTable = (OlapTable) table;
-                    Partition partition = olapTable.getPartition(partitionId);
-                    if (partition == null) {
-                        isSync = false;
-                        break;
-                    }
-                    partitionName = partition.getName();
-
-                    MaterializedIndex index = partition.getIndex(indexId);
-                    if (index == null) {
-                        isSync = false;
-                        break;
-                    }
-                    indexName = olapTable.getIndexNameById(indexId);
-
-                    Tablet tablet = index.getTablet(tabletId);
-                    if (tablet == null) {
-                        isSync = false;
-                        break;
-                    }
-
-                    tabletIdx = index.getTabletOrderIdx(tablet.getId());
-
-                    List<Replica> replicas = tablet.getReplicas();
-                    for (Replica replica : replicas) {
-                        Replica tmp = invertedIndex.getReplica(tabletId, 
replica.getBackendIdWithoutException());
-                        if (tmp == null) {
-                            isSync = false;
-                            break;
-                        }
-                        // use !=, not equals(), because this should be the 
same object.
-                        if (tmp != replica) {
-                            isSync = false;
-                            break;
-                        }
-                    }
-
-                } finally {
-                    table.readUnlock();
-                }
-            } while (false);
-
-            String detailCmd = String.format("SHOW PROC 
'/dbs/%d/%d/partitions/%d/%d/%d';",
-                    dbId, tableId, partitionId, indexId, tabletId);
-            rows.add(Lists.newArrayList(dbName, tableName, partitionName, 
indexName,
-                    dbId.toString(), tableId.toString(),
-                    partitionId.toString(), indexId.toString(),
-                    isSync.toString(), String.valueOf(tabletIdx), 
String.valueOf(queryHits), detailCmd));
-        } else {
-            Database db = 
env.getInternalCatalog().getDbOrAnalysisException(showStmt.getDbName());
-            OlapTable olapTable = 
db.getOlapTableOrAnalysisException(showStmt.getTableName());
-
-            olapTable.readLock();
-            try {
-                long sizeLimit = -1;
-                if (showStmt.hasOffset() && showStmt.hasLimit()) {
-                    sizeLimit = showStmt.getOffset() + showStmt.getLimit();
-                } else if (showStmt.hasLimit()) {
-                    sizeLimit = showStmt.getLimit();
-                }
-                boolean stop = false;
-                Collection<Partition> partitions = new ArrayList<Partition>();
-                if (showStmt.hasPartition()) {
-                    PartitionNames partitionNames = 
showStmt.getPartitionNames();
-                    for (String partName : partitionNames.getPartitionNames()) 
{
-                        Partition partition = olapTable.getPartition(partName, 
partitionNames.isTemp());
-                        if (partition == null) {
-                            throw new AnalysisException("Unknown partition: " 
+ partName);
-                        }
-                        partitions.add(partition);
-                    }
-                } else {
-                    partitions = olapTable.getPartitions();
-                }
-                List<List<Comparable>> tabletInfos = new ArrayList<>();
-                String indexName = showStmt.getIndexName();
-                long indexId = -1;
-                if (indexName != null) {
-                    Long id = olapTable.getIndexIdByName(indexName);
-                    if (id == null) {
-                        // invalid indexName
-                        
ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_TABLE, 
showStmt.getIndexName(),
-                                showStmt.getDbName());
-                    }
-                    indexId = id;
-                }
-                for (Partition partition : partitions) {
-                    if (stop) {
-                        break;
-                    }
-                    for (MaterializedIndex index : 
partition.getMaterializedIndices(IndexExtState.ALL)) {
-                        if (indexId > -1 && index.getId() != indexId) {
-                            continue;
-                        }
-                        TabletsProcDir procDir = new TabletsProcDir(olapTable, 
index);
-                        tabletInfos.addAll(procDir.fetchComparableResult(
-                                showStmt.getVersion(), 
showStmt.getBackendId(), showStmt.getReplicaState()));
-                        if (sizeLimit > -1 && tabletInfos.size() >= sizeLimit) 
{
-                            stop = true;
-                            break;
-                        }
-                    }
-                }
-                if (showStmt.hasOffset() && showStmt.getOffset() >= 
tabletInfos.size()) {
-                    tabletInfos.clear();
-                } else {
-                    // order by
-                    List<OrderByPair> orderByPairs = 
showStmt.getOrderByPairs();
-                    ListComparator<List<Comparable>> comparator = null;
-                    if (orderByPairs != null) {
-                        OrderByPair[] orderByPairArr = new 
OrderByPair[orderByPairs.size()];
-                        comparator = new 
ListComparator<>(orderByPairs.toArray(orderByPairArr));
-                    } else {
-                        // order by tabletId, replicaId
-                        comparator = new ListComparator<>(0, 1);
-                    }
-                    Collections.sort(tabletInfos, comparator);
-                    if (sizeLimit > -1) {
-                        tabletInfos = tabletInfos.subList((int) 
showStmt.getOffset(),
-                                Math.min((int) sizeLimit, tabletInfos.size()));
-                    }
-
-                    for (List<Comparable> tabletInfo : tabletInfos) {
-                        List<String> oneTablet = new 
ArrayList<String>(tabletInfo.size());
-                        for (Comparable column : tabletInfo) {
-                            oneTablet.add(column.toString());
-                        }
-                        rows.add(oneTablet);
-                    }
-                }
-            } finally {
-                olapTable.readUnlock();
-            }
-        }
-
-        resultSet = new ShowResultSet(showStmt.getMetaData(), rows);
-    }
-
     // Handle show brokers
     private void handleShowBroker() {
         ShowBrokerStmt showStmt = (ShowBrokerStmt) stmt;
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
index af8d149324b..a67ff754c4b 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
@@ -24,7 +24,6 @@ import org.apache.doris.analysis.CreateTableStmt;
 import org.apache.doris.analysis.RecoverPartitionStmt;
 import org.apache.doris.analysis.ShowPartitionsStmt;
 import org.apache.doris.analysis.ShowStmt;
-import org.apache.doris.analysis.ShowTabletStmt;
 import org.apache.doris.analysis.TruncateTableStmt;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.common.AnalysisException;
@@ -33,10 +32,14 @@ import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.jmockit.Deencapsulation;
 import org.apache.doris.meta.MetaContext;
+import 
org.apache.doris.nereids.trees.plans.commands.ShowTabletsFromTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.info.PartitionNamesInfo;
+import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.ShowExecutor;
 import org.apache.doris.qe.ShowResultSet;
+import org.apache.doris.qe.StmtExecutor;
 import org.apache.doris.utframe.UtFrameUtils;
 
 import com.google.common.base.Joiner;
@@ -54,6 +57,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -117,10 +121,13 @@ public class TempPartitionTest {
 
     private List<List<String>> checkTablet(String tbl, String partitions, 
boolean isTemp, int expected)
             throws Exception {
-        String showStr = "show tablets from " + tbl + (isTemp ? " temporary" : 
"") + " partition (" + partitions + ");";
-        ShowTabletStmt showStmt = (ShowTabletStmt) 
UtFrameUtils.parseAndAnalyzeStmt(showStr, ctx);
-        ShowExecutor executor = new ShowExecutor(ctx, (ShowStmt) showStmt);
-        ShowResultSet showResultSet = executor.execute();
+        // show tablets from " + tbl + (isTemp ? " temporary" : "") + " 
partition (" + partitions + ")
+        TableNameInfo tableNameInfo = new 
TableNameInfo(Arrays.asList(tbl.split("\\.")));
+        PartitionNamesInfo partitionNamesInfo = new PartitionNamesInfo(isTemp,
+                Arrays.asList(partitions.split(",")));
+        ShowTabletsFromTableCommand command = new 
ShowTabletsFromTableCommand(tableNameInfo, partitionNamesInfo,
+                null, null, 5, 0);
+        ShowResultSet showResultSet = command.doRun(ctx, new StmtExecutor(ctx, 
""));
         List<List<String>> rows = showResultSet.getResultRows();
         if (expected != -1) {
             Assert.assertEquals(expected, rows.size());
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java
index 346245de4ec..ddf1b3ce9b9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TruncateTableTest.java
@@ -20,17 +20,18 @@ package org.apache.doris.catalog;
 import org.apache.doris.analysis.AlterTableStmt;
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableStmt;
-import org.apache.doris.analysis.ShowStmt;
-import org.apache.doris.analysis.ShowTabletStmt;
 import org.apache.doris.analysis.TruncateTableStmt;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.common.util.DebugPointUtil;
 import org.apache.doris.common.util.DebugPointUtil.DebugPoint;
+import 
org.apache.doris.nereids.trees.plans.commands.ShowTabletsFromTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.info.PartitionNamesInfo;
+import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo;
 import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.qe.ShowExecutor;
 import org.apache.doris.qe.ShowResultSet;
+import org.apache.doris.qe.StmtExecutor;
 import org.apache.doris.utframe.UtFrameUtils;
 
 import com.google.common.collect.Maps;
@@ -41,6 +42,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -232,10 +234,13 @@ public class TruncateTableTest {
     }
 
     private List<List<String>> checkShowTabletResultNum(String tbl, String 
partition, int expected) throws Exception {
-        String showStr = "show tablets from " + tbl + " partition(" + 
partition + ")";
-        ShowTabletStmt showStmt = (ShowTabletStmt) 
UtFrameUtils.parseAndAnalyzeStmt(showStr, connectContext);
-        ShowExecutor executor = new ShowExecutor(connectContext, (ShowStmt) 
showStmt);
-        ShowResultSet showResultSet = executor.execute();
+        // "show tablets from " + tbl + " partition(" + partition + ")
+        TableNameInfo tableNameInfo = new 
TableNameInfo(Arrays.asList(tbl.split("\\.")));
+        PartitionNamesInfo partitionNamesInfo = new PartitionNamesInfo(false,
+                Arrays.asList(partition.split(",")));
+        ShowTabletsFromTableCommand command = new 
ShowTabletsFromTableCommand(tableNameInfo, partitionNamesInfo,
+                null, null, 5, 0);
+        ShowResultSet showResultSet = command.doRun(connectContext, new 
StmtExecutor(connectContext, ""));
         List<List<String>> rows = showResultSet.getResultRows();
         Assert.assertEquals(expected, rows.size());
         return rows;


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to