You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/04/08 02:31:20 UTC

[incubator-doris] 06/14: [feature](diagnose) support "ADMIN DIAGNOSE TABLET" stmt (#8839)

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

morningman pushed a commit to branch dev-1.0.1
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit cf94d9fe0f362db70dbf0deadc500bc94913799d
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Thu Apr 7 11:30:03 2022 +0800

    [feature](diagnose) support "ADMIN DIAGNOSE TABLET" stmt (#8839)
    
    `ADMIN DIAGNOSE TABLET tablet_id`
    
    This statement makes it easier to quickly diagnose the status of a tablet.
    See "ADMIN-DIAGNOSE-TABLET.md" for details
    
    ```
    mysql> admin diagnose tablet 10196;
    +----------------------------------+------------------------------+------------+
    | Item                             | Info                         | Suggestion |
    +----------------------------------+------------------------------+------------+
    | TabletExist                      | Yes                          |            |
    | TabletId                         | 10196                        |            |
    | Database                         | default_cluster:db1: 10192   |            |
    | Table                            | tbl1: 10194                  |            |
    | Partition                        | tbl1: 10193                  |            |
    | MaterializedIndex                | tbl1: 10195                  |            |
    | Replicas(ReplicaId -> BackendId) | {"10197":10002}              |            |
    | ReplicasNum                      | OK                           |            |
    | ReplicaBackendStatus             | Backend 10002 is not alive.  |            |
    | ReplicaVersionStatus             | OK                           |            |
    | ReplicaStatus                    | OK                           |            |
    | ReplicaCompactionStatus          | OK                           |            |
    +----------------------------------+------------------------------+------------+
    ```
---
 docs/.vuepress/sidebar/en.js                       |   1 +
 docs/.vuepress/sidebar/zh-CN.js                    |   1 +
 .../Administration/ADMIN-DIAGNOSE-TABLET.md        |  59 ++++++++
 .../Administration/ADMIN-DIAGNOSE-TABLET.md        |  59 ++++++++
 fe/fe-core/pom.xml                                 |   4 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |   8 +-
 .../doris/analysis/AdminDiagnoseTabletStmt.java    |  76 ++++++++++
 .../java/org/apache/doris/qe/ShowExecutor.java     |  11 ++
 .../java/org/apache/doris/system/Diagnoser.java    | 163 +++++++++++++++++++++
 fe/fe-core/src/main/jflex/sql_scanner.flex         |   1 +
 .../doris/clone/TabletReplicaTooSlowTest.java      |   8 +
 11 files changed, 388 insertions(+), 3 deletions(-)

diff --git a/docs/.vuepress/sidebar/en.js b/docs/.vuepress/sidebar/en.js
index 896805f6c2..2361963495 100644
--- a/docs/.vuepress/sidebar/en.js
+++ b/docs/.vuepress/sidebar/en.js
@@ -588,6 +588,7 @@ module.exports = [
               "ADMIN SHOW CONFIG",
               "ADMIN SHOW REPLICA DISTRIBUTION",
               "ADMIN SHOW REPLICA STATUS",
+              "ADMIN-DIAGNOSE-TABLET",
               "ALTER CLUSTER",
               "ALTER SYSTEM",
               "CANCEL DECOMMISSION",
diff --git a/docs/.vuepress/sidebar/zh-CN.js b/docs/.vuepress/sidebar/zh-CN.js
index 9aa410753b..dcb4a87597 100644
--- a/docs/.vuepress/sidebar/zh-CN.js
+++ b/docs/.vuepress/sidebar/zh-CN.js
@@ -590,6 +590,7 @@ module.exports = [
               "ADMIN SHOW CONFIG",
               "ADMIN SHOW REPLICA DISTRIBUTION",
               "ADMIN SHOW REPLICA STATUS",
+              "ADMIN-DIAGNOSE-TABLET",
               "ALTER CLUSTER",
               "ALTER SYSTEM",
               "CANCEL DECOMMISSION",
diff --git a/docs/en/sql-reference/sql-statements/Administration/ADMIN-DIAGNOSE-TABLET.md b/docs/en/sql-reference/sql-statements/Administration/ADMIN-DIAGNOSE-TABLET.md
new file mode 100644
index 0000000000..e41bf98c53
--- /dev/null
+++ b/docs/en/sql-reference/sql-statements/Administration/ADMIN-DIAGNOSE-TABLET.md
@@ -0,0 +1,59 @@
+---
+{
+    "title": "ADMIN DIAGNOSE TABLET",
+    "language": "en"
+}
+---
+
+<!-- 
+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.
+-->
+
+# ADMIN DIAGNOSE TABLET
+## description
+
+    This statement is used to diagnose the specified tablet. The results will show information about the tablet and some potential problems.
+
+    grammar:
+
+        ADMIN DIAGNOSE TABLET tblet_id
+
+    illustrate:
+
+        The lines of information in the result are as follows:
+        1. TabletExist:                         Whether the Tablet exists
+        2. TabletId:                            Tablet ID
+        3. Database:                            The DB to which the Tablet belongs and its ID
+        4. Table:                               The Table to which Tablet belongs and its ID
+        5. Partition:                           The Partition to which the Tablet belongs and its ID
+        6. MaterializedIndex:                   The materialized view to which the Tablet belongs and its ID
+        7. Replicas(ReplicaId -> BackendId):    Tablet replicas and their BE.
+        8. ReplicasNum:                         Whether the number of replicas is correct.
+        9. ReplicaBackendStatus:                Whether the BE node where the replica is located is normal.
+        10.ReplicaVersionStatus:                Whether the version number of the replica is normal.
+        11.ReplicaStatus:                       Whether the replica status is normal.
+        12.ReplicaCompactionStatus:             Whether the replica Compaction status is normal.
+        
+## example
+
+    1. Diagnose tablet 10001
+
+        ADMIN DIAGNOSE TABLET 10001;
+        
+## keyword
+    ADMIN,DIAGNOSE,TABLET
diff --git a/docs/zh-CN/sql-reference/sql-statements/Administration/ADMIN-DIAGNOSE-TABLET.md b/docs/zh-CN/sql-reference/sql-statements/Administration/ADMIN-DIAGNOSE-TABLET.md
new file mode 100644
index 0000000000..74d6e6699a
--- /dev/null
+++ b/docs/zh-CN/sql-reference/sql-statements/Administration/ADMIN-DIAGNOSE-TABLET.md
@@ -0,0 +1,59 @@
+---
+{
+    "title": "ADMIN DIAGNOSE TABLET",
+    "language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+# ADMIN DIAGNOSE TABLET
+## description
+
+    该语句用于诊断指定 tablet。结果中将显示这个 tablet 的信息和一些潜在的问题。
+
+    语法:
+
+        ADMIN DIAGNOSE TABLET tblet_id
+
+    说明:
+
+        结果中的各行信息如下:
+        1. TabletExist:                         Tablet是否存在
+        2. TabletId:                            Tablet ID
+        3. Database:                            Tablet 所属 DB 和其 ID
+        4. Table:                               Tablet 所属 Table 和其 ID
+        5. Partition:                           Tablet 所属 Partition 和其 ID
+        6. MaterializedIndex:                   Tablet 所属物化视图和其 ID
+        7. Replicas(ReplicaId -> BackendId):    Tablet 各副本和其所在 BE。
+        8. ReplicasNum:                         副本数量是否正确。
+        9. ReplicaBackendStatus:                副本所在 BE 节点是否正常。
+        10.ReplicaVersionStatus:                副本的版本号是否正常。
+        11.ReplicaStatus:                       副本状态是否正常。
+        12.ReplicaCompactionStatus:             副本 Compaction 状态是否正常。
+        
+## example
+
+    1. 查看 Tablet 10001 的诊断结果
+
+        ADMIN DIAGNOSE TABLET 10001;
+        
+## keyword
+    ADMIN,DIAGNOSE,TABLET
diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml
index 7a8fe89ef5..2f271b6ad9 100644
--- a/fe/fe-core/pom.xml
+++ b/fe/fe-core/pom.xml
@@ -688,9 +688,9 @@ under the License.
                             <goal>run</goal>
                         </goals>
                         <configuration>
-                            <protocCommand>${doris.thirdparty}/installed/bin/protoc</protocCommand>
+                            <!--protocCommand>${doris.thirdparty}/installed/bin/protoc</protocCommand-->
                             <!-->You can use following protocArtifact instead of protocCommand, so that you don't need to install protobuf tools<-->
-                            <!--protocArtifact>com.google.protobuf:protoc:${protobuf.version}</protocArtifact-->
+                            <protocArtifact>com.google.protobuf:protoc:${protobuf.version}</protocArtifact>
                             <protocVersion>${protobuf.version}</protocVersion>
                             <inputDirectories>
                                 <include>${doris.home}/gensrc/proto</include>
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index f42e864734..9e288b2fc2 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -241,7 +241,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALIAS, KW_ALL, KW_A
     KW_COLLATE, KW_COLLATION, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMMIT, KW_COMMITTED, KW_COMPACT,
     KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_CONVERT, KW_COUNT, KW_CREATE, KW_CREATION, KW_CROSS, KW_CUBE, KW_CURRENT, KW_CURRENT_USER,
     KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DAY, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE,
-    KW_DELETE, KW_UPDATE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_DYNAMIC, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE,
+    KW_DELETE, KW_UPDATE, KW_DIAGNOSE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_DYNAMIC, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE,
     KW_ELSE, KW_ENABLE, KW_ENCRYPTKEY, KW_ENCRYPTKEYS, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXCEPT, KW_EXCLUDE,
     KW_EXISTS, KW_EXPORT, KW_EXTENDED, KW_EXTERNAL, KW_EXTRACT,
     KW_FALSE, KW_FEATURE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FIELDS, KW_FILE, KW_FILTER, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORCE, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, KW_FUNCTIONS,
@@ -5283,6 +5283,10 @@ admin_stmt ::=
     {:
         RESULT = new AdminCleanTrashStmt(null);
     :}
+    | KW_ADMIN KW_DIAGNOSE KW_TABLET INTEGER_LITERAL:tabletId
+    {:
+        RESULT = new AdminDiagnoseTabletStmt(tabletId);
+    :}
     ;
 
 truncate_stmt ::=
@@ -5458,6 +5462,8 @@ keyword ::=
     {: RESULT = id; :}
     | KW_DECIMAL:id
     {: RESULT = id; :}
+    | KW_DIAGNOSE:id
+    {: RESULT = id; :}
     | KW_DISTINCTPC:id
     {: RESULT = id; :}
     | KW_DISTINCTPCSA:id
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminDiagnoseTabletStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminDiagnoseTabletStmt.java
new file mode 100644
index 0000000000..38dae3fbd7
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminDiagnoseTabletStmt.java
@@ -0,0 +1,76 @@
+// 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.Catalog;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.UserException;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.ShowResultSetMetaData;
+
+import com.google.common.collect.ImmutableList;
+
+// ADMIN DIAGNOSE TABLET tablet_id
+public class AdminDiagnoseTabletStmt extends ShowStmt {
+    public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>()
+            .add("Item").add("Info").add("Suggestion")
+            .build();
+
+    private long tabletId;
+
+    public AdminDiagnoseTabletStmt(long tabletId) {
+        this.tabletId = tabletId;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws UserException {
+        super.analyze(analyzer);
+
+        // check auth
+        if (!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) {
+            ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN");
+        }
+    }
+
+    public long getTabletId() {
+        return tabletId;
+    }
+
+    @Override
+    public String toSql() {
+        return "ADMIN DIAGNOSE TABLET " + tabletId;
+    }
+
+    @Override
+    public ShowResultSetMetaData getMetaData() {
+        ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder();
+        for (String title : TITLE_NAMES) {
+            builder.addColumn(new Column(title, ScalarType.createVarchar(1024)));
+        }
+        return builder.build();
+    }
+
+    @Override
+    public RedirectStatus getRedirectStatus() {
+        return RedirectStatus.FORWARD_NO_SYNC;
+    }
+}
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 330b816dec..5fb6ade642 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
@@ -17,6 +17,7 @@
 
 package org.apache.doris.qe;
 
+import org.apache.doris.analysis.AdminDiagnoseTabletStmt;
 import org.apache.doris.analysis.AdminShowConfigStmt;
 import org.apache.doris.analysis.AdminShowReplicaDistributionStmt;
 import org.apache.doris.analysis.AdminShowReplicaStatusStmt;
@@ -155,6 +156,7 @@ import org.apache.doris.load.LoadJob.JobState;
 import org.apache.doris.load.routineload.RoutineLoadJob;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.system.Backend;
+import org.apache.doris.system.Diagnoser;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TUnit;
 import org.apache.doris.transaction.GlobalTransactionMgr;
@@ -333,6 +335,8 @@ public class ShowExecutor {
             handleShowTableCreation();
         } else if (stmt instanceof ShowLastInsertStmt) {
             handleShowLastInsert();
+        } else if (stmt instanceof AdminDiagnoseTabletStmt) {
+            handleAdminDiagnoseTablet();
         } else {
             handleEmtpy();
         }
@@ -2128,4 +2132,11 @@ public class ShowExecutor {
         resultSet = new ShowResultSet(showMetaData, resultRowSet);
     }
 
+    private void handleAdminDiagnoseTablet() {
+        AdminDiagnoseTabletStmt showStmt = (AdminDiagnoseTabletStmt) stmt;
+        List<List<String>> resultRowSet = Diagnoser.diagnoseTablet(showStmt.getTabletId());
+        ShowResultSetMetaData showMetaData = showStmt.getMetaData();
+        resultSet = new ShowResultSet(showMetaData, resultRowSet);
+    }
+
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Diagnoser.java b/fe/fe-core/src/main/java/org/apache/doris/system/Diagnoser.java
new file mode 100644
index 0000000000..24048ac251
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/Diagnoser.java
@@ -0,0 +1,163 @@
+// 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.system;
+
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+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.Tablet;
+import org.apache.doris.catalog.TabletInvertedIndex;
+import org.apache.doris.catalog.TabletMeta;
+import org.apache.doris.common.Config;
+
+import com.google.common.collect.Lists;
+
+import org.json.simple.JSONObject;
+
+import java.util.List;
+
+// This is a util class to diagnose the Doris system
+public class Diagnoser {
+    // To diagnose a given tablet and return the info and issues about it
+    // - tablet exist:
+    // - tablet id
+    // - database
+    // - table
+    // - partition
+    // - materialized view
+    // - replica info: {"replica_id" : "backend id"}
+    // - replica num
+    // - ReplicaBackendStatus
+    // - ReplicaVersionStatus
+    // - ReplicaStatus
+    // - ReplicaCompactionStatus
+    //
+    public static List<List<String>> diagnoseTablet(long tabletId) {
+        List<List<String>> results = Lists.newArrayList();
+        TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
+        TabletMeta tabletMeta = invertedIndex.getTabletMeta(tabletId);
+        if (tabletMeta == null) {
+            results.add(Lists.newArrayList("TabletExist", "No", ""));
+            return results;
+        }
+        results.add(Lists.newArrayList("TabletExist", "Yes", ""));
+        results.add(Lists.newArrayList("TabletId", String.valueOf(tabletId), ""));
+        // database
+        Database db = Catalog.getCurrentCatalog().getDbNullable(tabletMeta.getDbId());
+        if (db == null) {
+            results.add(Lists.newArrayList("Database", "Not exist", ""));
+            return results;
+        }
+        results.add(Lists.newArrayList("Database", db.getFullName() + ": " + db.getId(), ""));
+        // table
+        OlapTable tbl = (OlapTable) db.getTableNullable(tabletMeta.getTableId());
+        if (tbl == null) {
+            results.add(Lists.newArrayList("Table", "Not exist", ""));
+            return results;
+        }
+        results.add(Lists.newArrayList("Table", tbl.getName() + ": " + tbl.getId(), ""));
+        // partition
+        Partition partition = tbl.getPartition(tabletMeta.getPartitionId());
+        if (partition == null) {
+            results.add(Lists.newArrayList("Partition", "Not exist", ""));
+            return results;
+        }
+        results.add(Lists.newArrayList("Partition", partition.getName() + ": " + partition.getId(), ""));
+        // materialized index
+        MaterializedIndex mIndex = partition.getIndex(tabletMeta.getIndexId());
+        if (mIndex == null) {
+            results.add(Lists.newArrayList("MaterializedIndex", "Not exist", ""));
+            return results;
+        }
+        results.add(Lists.newArrayList("MaterializedIndex", tbl.getIndexNameById(mIndex.getId()) + ": " + mIndex.getId(), ""));
+        // replica info
+        Tablet tablet = mIndex.getTablet(tabletId);
+        List<Replica> replicas = tablet.getReplicas();
+        JSONObject jobj = new JSONObject();
+        for (Replica replica : replicas) {
+            jobj.put(replica.getId(), replica.getBackendId());
+        }
+        results.add(Lists.newArrayList("Replicas(ReplicaId -> BackendId)", jobj.toJSONString(), ""));
+        // replica
+        short replicaNum = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
+        if (replicas.size() != replicaNum) {
+            results.add(Lists.newArrayList("ReplicasNum", "Replica num is " + replicas.size() + ", expected: " + replicaNum, ""));
+        } else {
+            results.add(Lists.newArrayList("ReplicasNum", "OK", ""));
+        }
+
+        SystemInfoService infoService = Catalog.getCurrentSystemInfo();
+        StringBuilder backendErr = new StringBuilder();
+        StringBuilder versionErr = new StringBuilder();
+        StringBuilder statusErr = new StringBuilder();
+        StringBuilder compactionErr = new StringBuilder();
+        for (Replica replica : replicas) {
+            // backend
+            do {
+                Backend be = infoService.getBackend(replica.getBackendId());
+                if (be == null) {
+                    backendErr.append("Backend " + replica.getBackendId() + " does not exist. ");
+                    break;
+                }
+                if (!be.isAlive()) {
+                    backendErr.append("Backend " + replica.getBackendId() + " is not alive. ");
+                    break;
+                }
+                if (be.isDecommissioned()) {
+                    backendErr.append("Backend " + replica.getBackendId() + " is decommission. ");
+                    break;
+                }
+                if (!be.isLoadAvailable()) {
+                    backendErr.append("Backend " + replica.getBackendId() + " is not load available. ");
+                    break;
+                }
+                if (!be.isQueryAvailable()) {
+                    backendErr.append("Backend " + replica.getBackendId() + " is not query available. ");
+                    break;
+                }
+                if (be.diskExceedLimit()) {
+                    backendErr.append("Backend " + replica.getBackendId() + " has no space left. ");
+                    break;
+                }
+            } while (false);
+            // version
+            if (replica.getVersion() != partition.getVisibleVersion()) {
+                versionErr.append("Replica on backend " + replica.getBackendId() + "'s version (" +
+                        replica.getVersion() + ") does not equal" +
+                        " to partition visible version (" + partition.getVisibleVersion() + ")");
+            }
+            // status
+            if (!replica.isAlive()) {
+                statusErr.append("Replica on backend " + replica.getBackendId() + "'s state is " + replica.getState()
+                        + ", and is bad: " + (replica.isBad() ? "Yes" : "No"));
+            }
+            if (replica.getVersionCount() > Config.min_version_count_indicate_replica_compaction_too_slow) {
+                compactionErr.append("Replica on backend " + replica.getBackendId() + "'s version count is too high: "
+                        + replica.getVersionCount());
+            }
+        }
+        results.add(Lists.newArrayList("ReplicaBackendStatus", (backendErr.length() == 0 ? "OK" : backendErr.toString()), ""));
+        results.add(Lists.newArrayList("ReplicaVersionStatus", (versionErr.length() == 0 ? "OK" : versionErr.toString()), ""));
+        results.add(Lists.newArrayList("ReplicaStatus", (statusErr.length() == 0 ? "OK" : statusErr.toString()), ""));
+        results.add(Lists.newArrayList("ReplicaCompactionStatus", (compactionErr.length() == 0 ? "OK" : compactionErr.toString()), ""));
+        return results;
+    }
+}
diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex
index 6f356569ce..1fb4a9d47a 100644
--- a/fe/fe-core/src/main/jflex/sql_scanner.flex
+++ b/fe/fe-core/src/main/jflex/sql_scanner.flex
@@ -162,6 +162,7 @@ import org.apache.doris.qe.SqlModeHelper;
         keywordMap.put("delete", new Integer(SqlParserSymbols.KW_DELETE));
         keywordMap.put("desc", new Integer(SqlParserSymbols.KW_DESC));
         keywordMap.put("describe", new Integer(SqlParserSymbols.KW_DESCRIBE));
+        keywordMap.put("diagnose", new Integer(SqlParserSymbols.KW_DIAGNOSE));
         keywordMap.put("distinct", new Integer(SqlParserSymbols.KW_DISTINCT));
         keywordMap.put("distinctpc", new Integer(SqlParserSymbols.KW_DISTINCTPC));
         keywordMap.put("distinctpc", new Integer(SqlParserSymbols.KW_DISTINCTPC));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java
index 2a48e64065..bc88fe94ff 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java
@@ -29,6 +29,7 @@ import org.apache.doris.common.FeConstants;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
+import org.apache.doris.system.Diagnoser;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TDisk;
 import org.apache.doris.thrift.TStorageMedium;
@@ -132,7 +133,9 @@ public class TabletReplicaTooSlowTest {
     private static void updateReplicaVersionCount() {
         Table<Long, Long, Replica> replicaMetaTable = Catalog.getCurrentInvertedIndex().getReplicaMetaTable();
         int versionCount = 1;
+        long tabletId = -1;
         for (Table.Cell<Long, Long, Replica> cell : replicaMetaTable.cellSet()) {
+            tabletId = cell.getRowKey();
             long beId = cell.getColumnKey();
             Backend be = Catalog.getCurrentSystemInfo().getBackend(beId);
             List<Long> pathHashes = be.getDisks().values().stream().map(DiskInfo::getPathHash).collect(Collectors.toList());
@@ -145,6 +148,10 @@ public class TabletReplicaTooSlowTest {
 
             replica.setPathHash(pathHashes.get(0));
         }
+
+        List<List<String>> result = Diagnoser.diagnoseTablet(tabletId);
+        Assert.assertEquals(12, result.size());
+        Assert.assertTrue(result.get(11).get(1).contains("version count is too high"));
     }
 
     @Test
@@ -158,6 +165,7 @@ public class TabletReplicaTooSlowTest {
                 "    \"replication_num\" = \"3\"\n" +
                 ")";
         ExceptionChecker.expectThrowsNoException(() -> createTable(createStr));
+
         int maxLoop = 300;
         boolean delete = false;
         while (maxLoop-- > 0) {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org