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 5fd21da747e7dd..bb79cd059f359e 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 @@ -290,6 +290,7 @@ supportedShowStatement | SHOW REPLICA DISTRIBUTION FROM baseTableRef #showReplicaDistribution | SHOW FULL? TRIGGERS ((FROM | IN) database=multipartIdentifier)? wildWhere? #showTriggers | SHOW TABLET DIAGNOSIS tabletId=INTEGER_VALUE #showDiagnoseTablet + | SHOW TABLET tabletId=INTEGER_VALUE #showTabletId | SHOW FRONTENDS name=identifier? #showFrontends | SHOW DATABASE databaseId=INTEGER_VALUE #showDatabaseId | SHOW TABLE tableId=INTEGER_VALUE #showTableId @@ -369,7 +370,6 @@ unsupportedShowStatement sortClause? propertyClause? #showData | SHOW TEMPORARY? PARTITIONS FROM tableName=multipartIdentifier wildWhere? sortClause? limitClause? #showPartitions - | SHOW TABLET tabletId=INTEGER_VALUE #showTabletId | SHOW TABLETS FROM tableName=multipartIdentifier partitionSpec? wildWhere? sortClause? limitClause? #showTabletsFromTable | SHOW BACKUP ((FROM | IN) database=multipartIdentifier)? wildWhere? #showBackup 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 0432e06c335472..e5c21bee2126af 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 @@ -604,6 +604,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowSyncJobCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableCreationCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTabletIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletStorageFormatCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; @@ -4956,6 +4957,12 @@ public LogicalPlan visitAdminDiagnoseTablet(AdminDiagnoseTabletContext ctx) { return new ShowDiagnoseTabletCommand(tabletId); } + @Override + public LogicalPlan visitShowTabletId(DorisParser.ShowTabletIdContext ctx) { + long tabletId = Long.parseLong(ctx.INTEGER_VALUE().getText()); + return new ShowTabletIdCommand(tabletId); + } + @Override public LogicalPlan visitShowCreateTable(ShowCreateTableContext ctx) { List nameParts = visitMultipartIdentifier(ctx.name); 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 1c0cb26f0f0702..59099fb77dcbe6 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 @@ -253,6 +253,7 @@ public enum PlanType { SHOW_TABLE_ID_COMMAND, SHOW_TRASH_COMMAND, SHOW_TABLET_STORAGE_FORMAT_COMMAND, + SHOW_TABLET_ID_COMMAND, SHOW_TRIGGERS_COMMAND, SHOW_VARIABLES_COMMAND, SHOW_AUTHORS_COMMAND, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletIdCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletIdCommand.java new file mode 100644 index 00000000000000..80c3971ebed816 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowTabletIdCommand.java @@ -0,0 +1,206 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.commands; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.catalog.TabletInvertedIndex; +import org.apache.doris.catalog.TabletMeta; +import org.apache.doris.common.Config; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeConstants; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSet; +import org.apache.doris.qe.ShowResultSetMetaData; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.statistics.query.QueryStatsUtil; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * show tablet id command + */ +public class ShowTabletIdCommand extends ShowCommand { + private final long tabletId; + + /** + * constructor + */ + public ShowTabletIdCommand(long tabletId) { + super(PlanType.SHOW_TABLET_ID_COMMAND); + this.tabletId = tabletId; + } + + /** + * get meta for show tabletId + */ + public ShowResultSetMetaData getMetaData() { + ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); + 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))); + return builder.build(); + } + + /** + * handle show tablet + */ + public List> handleShowTablet() { + + List> rows = Lists.newArrayList(); + 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.getCurrentEnv().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 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)); + return rows; + } + + @Override + public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "SHOW TABLET"); + } + + // Set the result set and send it using the executor + return new ShowResultSet(getMetaData(), handleShowTablet()); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowTabletIdCommand(this, context); + } + + @Override + public RedirectStatus toRedirectStatus() { + 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/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 0f5cafa919227c..381dae74232fc3 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 @@ -137,6 +137,7 @@ import org.apache.doris.nereids.trees.plans.commands.ShowSyncJobCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableCreationCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTableIdCommand; +import org.apache.doris.nereids.trees.plans.commands.ShowTabletIdCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletStorageFormatCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTabletsBelongCommand; import org.apache.doris.nereids.trees.plans.commands.ShowTrashCommand; @@ -737,6 +738,10 @@ default R visitShowTabletStorageFormatCommand(ShowTabletStorageFormatCommand sho return visitCommand(showTabletStorageFormatCommand, context); } + default R visitShowTabletIdCommand(ShowTabletIdCommand showTabletIdCommand, C context) { + return visitCommand(showTabletIdCommand, context); + } + default R visitShowQueryProfileCommand(ShowQueryProfileCommand showQueryProfileCommand, C context) { return visitCommand(showQueryProfileCommand, context); diff --git a/regression-test/suites/nereids_p0/show/test_nereids_show_tablet_id.groovy b/regression-test/suites/nereids_p0/show/test_nereids_show_tablet_id.groovy new file mode 100644 index 00000000000000..2131a7cd225bd1 --- /dev/null +++ b/regression-test/suites/nereids_p0/show/test_nereids_show_tablet_id.groovy @@ -0,0 +1,39 @@ +// 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_tablet_id") { + String tableName = "test_tablet_id"; + String tabletId = ""; + try { + // Create a new table to test the SHOW TABLET command + sql "CREATE TABLE IF NOT EXISTS ${tableName} (id INT, name STRING) DUPLICATE KEY(id) DISTRIBUTED BY HASH(id) BUCKETS 3 PROPERTIES('replication_num'='1');" + + // Extract tablet ID from the created table + def showTabletsResult = sql "SHOW TABLETS FROM ${tableName}" + assert showTabletsResult.size() > 0 + tabletId = showTabletsResult[0][0] // Assuming the first tablet ID is used + + // Execute the SHOW TABLET command and verify the output + checkNereidsExecute("SHOW TABLET ${tabletId}") + } catch (Exception e) { + log.error("Failed to execute SHOW TABLET command", e) + throw e + } finally { + try_sql("DROP TABLE IF EXISTS ${tableName}") + } +} +