You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by zy...@apache.org on 2023/03/23 00:38:51 UTC

[iotdb] branch rc/1.1.0 updated (6616b0d1d3 -> 9fda3fb9f6)

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

zyk pushed a change to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git


    from 6616b0d1d3 [To rel/1.1][IOTDB-5639][compaction]Fix file not found exception in cross space compaction selector (#9255)
     new 8877512e66 [IOTDB-5701] Optimize the management of performance overview related metrics (#9381) (#9396)
     new 9f6e0e72bf [To rel/1.1] Correct CQ example in docs (#9407)
     new 0e5bd28b9c Seperate disk dashboard (#9413)
     new 9fda3fb9f6 Compatible with the client parameters on DataNode  (#9421)

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../consensus/iot/IoTConsensusServerImpl.java      |   11 +-
 .../ratis/ApplicationStateMachineProxy.java        |   18 +-
 .../iotdb/consensus/simple/SimpleConsensus.java    |   16 +-
 .../Administration-Management/Administration.md    |    2 +-
 .../Apache-IoTDB-ConfigNode-Dashboard.json         | 2586 ++++++++------------
 .../Apache-IoTDB-DataNode-Dashboard.json           | 1147 ++-------
 .../Apache-IoTDB-Disk-IO-Dashboard.json            |  894 +++++++
 .../Administration-Management/Administration.md    |    2 +-
 .../iotdb/metrics/metricsets/disk/DiskMetrics.java |   56 +-
 .../metric/enums/PerformanceOverviewMetrics.java   |  256 +-
 .../org/apache/iotdb/db/auth/AuthorityChecker.java |    7 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   12 +-
 .../iotdb/db/engine/storagegroup/DataRegion.java   |   16 +-
 .../db/engine/storagegroup/TsFileProcessor.java    |   25 +-
 .../execution/executor/RegionWriteExecutor.java    |   12 +-
 .../metric/PerformanceOverviewMetricsManager.java  |  214 --
 .../db/mpp/plan/execution/QueryExecution.java      |   11 +-
 .../db/mpp/plan/parser/StatementGenerator.java     |   54 +-
 .../plan/scheduler/AsyncSendPlanNodeHandler.java   |    8 +-
 .../scheduler/FragmentInstanceDispatcherImpl.java  |    6 +-
 .../db/service/metrics/DataNodeMetricsHelper.java  |    4 +
 21 files changed, 2403 insertions(+), 2954 deletions(-)
 create mode 100644 docs/UserGuide/Monitor-Alert/Apache-IoTDB-Disk-IO-Dashboard.json
 delete mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/metric/PerformanceOverviewMetricsManager.java


[iotdb] 01/04: [IOTDB-5701] Optimize the management of performance overview related metrics (#9381) (#9396)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 8877512e66b09d62a9ec5dbe3a0d97eefe1ff7f5
Author: ZhangHongYin <46...@users.noreply.github.com>
AuthorDate: Wed Mar 22 12:47:10 2023 +0800

    [IOTDB-5701] Optimize the management of performance overview related metrics (#9381) (#9396)
---
 .../consensus/iot/IoTConsensusServerImpl.java      |  11 +-
 .../ratis/ApplicationStateMachineProxy.java        |  18 +-
 .../iotdb/consensus/simple/SimpleConsensus.java    |  16 +-
 .../metric/enums/PerformanceOverviewMetrics.java   | 256 +++++++++++++++++++--
 .../org/apache/iotdb/db/auth/AuthorityChecker.java |   7 +-
 .../iotdb/db/engine/storagegroup/DataRegion.java   |  16 +-
 .../db/engine/storagegroup/TsFileProcessor.java    |  25 +-
 .../execution/executor/RegionWriteExecutor.java    |  12 +-
 .../metric/PerformanceOverviewMetricsManager.java  | 214 -----------------
 .../db/mpp/plan/execution/QueryExecution.java      |  11 +-
 .../db/mpp/plan/parser/StatementGenerator.java     |  54 ++---
 .../plan/scheduler/AsyncSendPlanNodeHandler.java   |   8 +-
 .../scheduler/FragmentInstanceDispatcherImpl.java  |   6 +-
 .../db/service/metrics/DataNodeMetricsHelper.java  |   4 +
 14 files changed, 326 insertions(+), 332 deletions(-)

diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java b/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java
index 3aadc12f57..201322fa48 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/iot/IoTConsensusServerImpl.java
@@ -94,6 +94,8 @@ public class IoTConsensusServerImpl {
   private static final String CONFIGURATION_TMP_FILE_NAME = "configuration.dat.tmp";
   public static final String SNAPSHOT_DIR_NAME = "snapshot";
   private static final Pattern SNAPSHOT_INDEX_PATTEN = Pattern.compile(".*[^\\d](?=(\\d+))");
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
   private final Logger logger = LoggerFactory.getLogger(IoTConsensusServerImpl.class);
   private final Peer thisNode;
   private final IStateMachine stateMachine;
@@ -224,14 +226,7 @@ public class IoTConsensusServerImpl {
       IConsensusRequest planNode = stateMachine.deserializeRequest(indexedConsensusRequest);
       long startWriteTime = System.nanoTime();
       TSStatus result = stateMachine.write(planNode);
-      MetricService.getInstance()
-          .timer(
-              System.nanoTime() - startWriteTime,
-              TimeUnit.NANOSECONDS,
-              Metric.PERFORMANCE_OVERVIEW_STORAGE_DETAIL.toString(),
-              MetricLevel.IMPORTANT,
-              Tag.STAGE.toString(),
-              PerformanceOverviewMetrics.ENGINE);
+      PERFORMANCE_OVERVIEW_METRICS.recordEngineCost(System.nanoTime() - startWriteTime);
 
       long writeToStateMachineEndTime = System.nanoTime();
       // statistic the time of writing request into stateMachine
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/ApplicationStateMachineProxy.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/ApplicationStateMachineProxy.java
index aacb0743be..80f11ac05e 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/ApplicationStateMachineProxy.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/ApplicationStateMachineProxy.java
@@ -20,16 +20,12 @@ package org.apache.iotdb.consensus.ratis;
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.service.metric.MetricService;
-import org.apache.iotdb.commons.service.metric.enums.Metric;
 import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
-import org.apache.iotdb.commons.service.metric.enums.Tag;
 import org.apache.iotdb.consensus.IStateMachine;
 import org.apache.iotdb.consensus.common.DataSet;
 import org.apache.iotdb.consensus.common.request.ByteBufferConsensusRequest;
 import org.apache.iotdb.consensus.common.request.IConsensusRequest;
 import org.apache.iotdb.consensus.ratis.metrics.RatisMetricsManager;
-import org.apache.iotdb.metrics.utils.MetricLevel;
 
 import org.apache.ratis.proto.RaftProtos;
 import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto;
@@ -58,7 +54,9 @@ import java.util.concurrent.TimeUnit;
 
 public class ApplicationStateMachineProxy extends BaseStateMachine {
 
-  private final Logger logger = LoggerFactory.getLogger(ApplicationStateMachineProxy.class);
+  private static final Logger logger = LoggerFactory.getLogger(ApplicationStateMachineProxy.class);
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
   private final IStateMachine applicationStateMachine;
   private final IStateMachine.RetryPolicy retryPolicy;
   private final SnapshotStorage snapshotStorage;
@@ -177,14 +175,8 @@ public class ApplicationStateMachineProxy extends BaseStateMachine {
     if (isLeader) {
       // only record time cost for data region in Performance Overview Dashboard
       if (consensusGroupType == TConsensusGroupType.DataRegion) {
-        MetricService.getInstance()
-            .timer(
-                System.nanoTime() - writeToStateMachineStartTime,
-                TimeUnit.NANOSECONDS,
-                Metric.PERFORMANCE_OVERVIEW_STORAGE_DETAIL.toString(),
-                MetricLevel.IMPORTANT,
-                Tag.STAGE.toString(),
-                PerformanceOverviewMetrics.ENGINE);
+        PERFORMANCE_OVERVIEW_METRICS.recordEngineCost(
+            System.nanoTime() - writeToStateMachineStartTime);
       }
       // statistic the time of write stateMachine
       RatisMetricsManager.getInstance()
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/simple/SimpleConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/simple/SimpleConsensus.java
index 27a1687e90..514b75c372 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/simple/SimpleConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/simple/SimpleConsensus.java
@@ -23,10 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.service.metric.MetricService;
-import org.apache.iotdb.commons.service.metric.enums.Metric;
 import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
-import org.apache.iotdb.commons.service.metric.enums.Tag;
 import org.apache.iotdb.commons.utils.FileUtils;
 import org.apache.iotdb.consensus.IConsensus;
 import org.apache.iotdb.consensus.IStateMachine;
@@ -41,7 +38,6 @@ import org.apache.iotdb.consensus.exception.ConsensusGroupAlreadyExistException;
 import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException;
 import org.apache.iotdb.consensus.exception.IllegalPeerEndpointException;
 import org.apache.iotdb.consensus.exception.IllegalPeerNumException;
-import org.apache.iotdb.metrics.utils.MetricLevel;
 import org.apache.iotdb.rpc.TSStatusCode;
 
 import org.slf4j.Logger;
@@ -56,7 +52,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -73,6 +68,8 @@ class SimpleConsensus implements IConsensus {
   private final File storageDir;
   private final IStateMachine.Registry registry;
   private final Map<ConsensusGroupId, SimpleServerImpl> stateMachineMap = new ConcurrentHashMap<>();
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
 
   public SimpleConsensus(ConsensusConfig config, Registry registry) {
     this.thisNode = config.getThisNodeEndPoint();
@@ -132,14 +129,7 @@ class SimpleConsensus implements IConsensus {
         long startWriteTime = System.nanoTime();
         status = impl.write(request);
         // only record time cost for data region in Performance Overview Dashboard
-        MetricService.getInstance()
-            .timer(
-                System.nanoTime() - startWriteTime,
-                TimeUnit.NANOSECONDS,
-                Metric.PERFORMANCE_OVERVIEW_STORAGE_DETAIL.toString(),
-                MetricLevel.IMPORTANT,
-                Tag.STAGE.toString(),
-                PerformanceOverviewMetrics.ENGINE);
+        PERFORMANCE_OVERVIEW_METRICS.recordEngineCost(System.nanoTime() - startWriteTime);
       } else {
         status = impl.write(request);
       }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/PerformanceOverviewMetrics.java b/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/PerformanceOverviewMetrics.java
index 9b59cefc7d..e82e5d7106 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/PerformanceOverviewMetrics.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/PerformanceOverviewMetrics.java
@@ -20,7 +20,9 @@
 package org.apache.iotdb.commons.service.metric.enums;
 
 import org.apache.iotdb.metrics.AbstractMetricService;
+import org.apache.iotdb.metrics.impl.DoNothingMetricManager;
 import org.apache.iotdb.metrics.metricsets.IMetricSet;
+import org.apache.iotdb.metrics.type.Timer;
 import org.apache.iotdb.metrics.utils.MetricInfo;
 import org.apache.iotdb.metrics.utils.MetricLevel;
 import org.apache.iotdb.metrics.utils.MetricType;
@@ -30,13 +32,20 @@ import java.util.Map;
 
 public class PerformanceOverviewMetrics implements IMetricSet {
   private static final Map<String, MetricInfo> metricInfoMap = new HashMap<>();
+
+  private PerformanceOverviewMetrics() {
+    // empty constructor
+  }
+
+  // region overview
+
   private static final String PERFORMANCE_OVERVIEW_DETAIL =
       Metric.PERFORMANCE_OVERVIEW_DETAIL.toString();
-  public static final String AUTHORITY = "authority";
-  public static final String PARSER = "parser";
-  public static final String ANALYZER = "analyzer";
-  public static final String PLANNER = "planner";
-  public static final String SCHEDULER = "scheduler";
+  private static final String AUTHORITY = "authority";
+  private static final String PARSER = "parser";
+  private static final String ANALYZER = "analyzer";
+  private static final String PLANNER = "planner";
+  private static final String SCHEDULER = "scheduler";
 
   static {
     metricInfoMap.put(
@@ -61,10 +70,45 @@ public class PerformanceOverviewMetrics implements IMetricSet {
             MetricType.TIMER, PERFORMANCE_OVERVIEW_DETAIL, Tag.STAGE.toString(), SCHEDULER));
   }
 
+  private Timer authTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer parseTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer analyzeTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer planTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer scheduleTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+
+  /** Record the time cost in authority stage. */
+  public void recordAuthCost(long costTimeInNanos) {
+    authTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost in parse stage. */
+  public void recordParseCost(long costTimeInNanos) {
+    parseTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost in analyze stage. */
+  public void recordAnalyzeCost(long costTimeInNanos) {
+    analyzeTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost in plan stage. */
+  public void recordPlanCost(long costTimeInNanos) {
+    planTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost in schedule stage. */
+  public void recordScheduleCost(long costTimeInNanos) {
+    scheduleTimer.updateNanos(costTimeInNanos);
+  }
+
+  // endregion
+
+  // region schedule
+
   private static final String PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL =
       Metric.PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL.toString();
-  public static final String LOCAL_SCHEDULE = "local_scheduler";
-  public static final String REMOTE_SCHEDULE = "remote_scheduler";
+  private static final String LOCAL_SCHEDULE = "local_scheduler";
+  private static final String REMOTE_SCHEDULE = "remote_scheduler";
 
   static {
     metricInfoMap.put(
@@ -83,11 +127,27 @@ public class PerformanceOverviewMetrics implements IMetricSet {
             REMOTE_SCHEDULE));
   }
 
+  private Timer localScheduleTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer remoteScheduleTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+
+  /** Record the time cost of local schedule. */
+  public void recordScheduleLocalCost(long costTimeInNanos) {
+    localScheduleTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of remote schedule. */
+  public void recordScheduleRemoteCost(long costTimeInNanos) {
+    remoteScheduleTimer.updateNanos(costTimeInNanos);
+  }
+
+  // endregion
+
+  // region local schedule
   private static final String PERFORMANCE_OVERVIEW_LOCAL_DETAIL =
       Metric.PERFORMANCE_OVERVIEW_LOCAL_DETAIL.toString();
-  public static final String SCHEMA_VALIDATE = "schema_validate";
-  public static final String TRIGGER = "trigger";
-  public static final String STORAGE = "storage";
+  private static final String SCHEMA_VALIDATE = "schema_validate";
+  private static final String TRIGGER = "trigger";
+  private static final String STORAGE = "storage";
 
   static {
     metricInfoMap.put(
@@ -107,9 +167,31 @@ public class PerformanceOverviewMetrics implements IMetricSet {
             MetricType.TIMER, PERFORMANCE_OVERVIEW_LOCAL_DETAIL, Tag.STAGE.toString(), STORAGE));
   }
 
+  private Timer schemaValidateTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer triggerTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer storageTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+
+  /** Record the time cost of schema validate stage in local schedule. */
+  public void recordScheduleSchemaValidateCost(long costTimeInNanos) {
+    schemaValidateTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of trigger stage in local schedule. */
+  public void recordScheduleTriggerCost(long costTimeInNanos) {
+    triggerTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of storage stage in local schedule. */
+  public void recordScheduleStorageCost(long costTimeInNanos) {
+    storageTimer.updateNanos(costTimeInNanos);
+  }
+
+  // endregion
+
+  // region storage
   private static final String PERFORMANCE_OVERVIEW_STORAGE_DETAIL =
       Metric.PERFORMANCE_OVERVIEW_STORAGE_DETAIL.toString();
-  public static final String ENGINE = "engine";
+  private static final String ENGINE = "engine";
 
   static {
     metricInfoMap.put(
@@ -118,14 +200,24 @@ public class PerformanceOverviewMetrics implements IMetricSet {
             MetricType.TIMER, PERFORMANCE_OVERVIEW_STORAGE_DETAIL, Tag.STAGE.toString(), ENGINE));
   }
 
+  private Timer engineTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+
+  public void recordEngineCost(long costTimeInNanos) {
+    engineTimer.updateNanos(costTimeInNanos);
+  }
+
+  // endregion
+
+  // region engine
+
   private static final String PERFORMANCE_OVERVIEW_ENGINE_DETAIL =
       Metric.PERFORMANCE_OVERVIEW_ENGINE_DETAIL.toString();
-  public static final String LOCK = "lock";
-  public static final String MEMORY_BLOCK = "memory_block";
-  public static final String CREATE_MEMTABLE_BLOCK = "create_memtable_block";
-  public static final String WAL = "wal";
-  public static final String MEMTABLE = "memtable";
-  public static final String LAST_CACHE = "last_cache";
+  private static final String LOCK = "lock";
+  private static final String MEMORY_BLOCK = "memory_block";
+  private static final String CREATE_MEMTABLE_BLOCK = "create_memtable_block";
+  private static final String WAL = "wal";
+  private static final String MEMTABLE = "memtable";
+  private static final String LAST_CACHE = "last_cache";
 
   static {
     metricInfoMap.put(
@@ -163,12 +255,121 @@ public class PerformanceOverviewMetrics implements IMetricSet {
             LAST_CACHE));
   }
 
+  private Timer lockTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer createMemtableBlockTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer memoryBlockTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer walTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer memtableTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+  private Timer lastCacheTimer = DoNothingMetricManager.DO_NOTHING_TIMER;
+
+  /** Record the time cost of lock in engine. */
+  public void recordScheduleLockCost(long costTimeInNanos) {
+    lockTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of create memtable block in engine. */
+  public void recordCreateMemtableBlockCost(long costTimeInNanos) {
+    createMemtableBlockTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of memory block in engine. */
+  public void recordScheduleMemoryBlockCost(long costTimeInNanos) {
+    memoryBlockTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of wal in engine. */
+  public void recordScheduleWalCost(long costTimeInNanos) {
+    walTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of memtable in engine. */
+  public void recordScheduleMemTableCost(long costTimeInNanos) {
+    memtableTimer.updateNanos(costTimeInNanos);
+  }
+
+  /** Record the time cost of update last cache in engine. */
+  public void recordScheduleUpdateLastCacheCost(long costTimeInNanos) {
+    lastCacheTimer.updateNanos(costTimeInNanos);
+  }
+
+  // endregion
+
   @Override
   public void bindTo(AbstractMetricService metricService) {
-    for (MetricInfo metricInfo : metricInfoMap.values()) {
-      metricService.getOrCreateTimer(
-          metricInfo.getName(), MetricLevel.CORE, metricInfo.getTagsInArray());
-    }
+    // bind overview metrics
+    authTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), AUTHORITY);
+    parseTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), PARSER);
+    analyzeTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), ANALYZER);
+    planTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), PLANNER);
+    scheduleTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), SCHEDULER);
+    // bind schedule metrics
+    localScheduleTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL,
+            MetricLevel.CORE,
+            Tag.STAGE.toString(),
+            LOCAL_SCHEDULE);
+    remoteScheduleTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL,
+            MetricLevel.CORE,
+            Tag.STAGE.toString(),
+            REMOTE_SCHEDULE);
+    // bind local schedule metrics
+    schemaValidateTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_LOCAL_DETAIL,
+            MetricLevel.CORE,
+            Tag.STAGE.toString(),
+            SCHEMA_VALIDATE);
+    triggerTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_LOCAL_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), TRIGGER);
+    storageTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_LOCAL_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), STORAGE);
+    // bind storage metrics
+    engineTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_STORAGE_DETAIL,
+            MetricLevel.CORE,
+            Tag.STAGE.toString(),
+            PerformanceOverviewMetrics.ENGINE);
+    // bind engine metrics
+    localScheduleTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_ENGINE_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), LOCK);
+    createMemtableBlockTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
+            MetricLevel.CORE,
+            Tag.STAGE.toString(),
+            CREATE_MEMTABLE_BLOCK);
+    memoryBlockTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
+            MetricLevel.CORE,
+            Tag.STAGE.toString(),
+            MEMORY_BLOCK);
+    walTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_ENGINE_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), WAL);
+    memtableTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_ENGINE_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), MEMTABLE);
+    lastCacheTimer =
+        metricService.getOrCreateTimer(
+            PERFORMANCE_OVERVIEW_ENGINE_DETAIL, MetricLevel.CORE, Tag.STAGE.toString(), LAST_CACHE);
   }
 
   @Override
@@ -177,4 +378,17 @@ public class PerformanceOverviewMetrics implements IMetricSet {
       metricService.remove(MetricType.TIMER, metricInfo.getName(), metricInfo.getTagsInArray());
     }
   }
+
+  private static class PerformanceOverviewMetricsHolder {
+
+    private static final PerformanceOverviewMetrics INSTANCE = new PerformanceOverviewMetrics();
+
+    private PerformanceOverviewMetricsHolder() {
+      // empty constructor
+    }
+  }
+
+  public static PerformanceOverviewMetrics getInstance() {
+    return PerformanceOverviewMetricsHolder.INSTANCE;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java b/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
index 62738656df..e802701b26 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
@@ -23,9 +23,9 @@ import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.db.conf.OperationType;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
@@ -49,6 +49,9 @@ public class AuthorityChecker {
 
   private static final AuthorizerManager authorizerManager = AuthorizerManager.getInstance();
 
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
+
   private AuthorityChecker() {
     // empty constructor
   }
@@ -122,7 +125,7 @@ public class AuthorityChecker {
       return onQueryException(
           e, OperationType.CHECK_AUTHORITY.getName(), TSStatusCode.EXECUTE_STATEMENT_ERROR);
     } finally {
-      PerformanceOverviewMetricsManager.recordAuthCost(System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordAuthCost(System.nanoTime() - startTime);
     }
     return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
index 9fe0e3346f..7aa38a3a68 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
@@ -32,6 +32,7 @@ import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.service.metric.MetricService;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.consensus.ConsensusFactory;
 import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -70,7 +71,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.cache.DataNodeSchemaCache;
 import org.apache.iotdb.db.metadata.idtable.IDTable;
 import org.apache.iotdb.db.metadata.idtable.IDTableManager;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
@@ -277,6 +277,8 @@ public class DataRegion implements IDataRegionForQuery {
 
   private final QueryMetricsManager queryMetricsManager = QueryMetricsManager.getInstance();
 
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
   /**
    * construct a database processor.
    *
@@ -924,7 +926,7 @@ public class DataRegion implements IDataRegionForQuery {
     }
     long startTime = System.nanoTime();
     writeLock("InsertRow");
-    PerformanceOverviewMetricsManager.recordScheduleLockCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleLockCost(System.nanoTime() - startTime);
     try {
       if (deleted) {
         return;
@@ -975,7 +977,7 @@ public class DataRegion implements IDataRegionForQuery {
     }
     long startTime = System.nanoTime();
     writeLock("insertTablet");
-    PerformanceOverviewMetricsManager.recordScheduleLockCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleLockCost(System.nanoTime() - startTime);
     try {
       if (deleted) {
         return;
@@ -1068,8 +1070,7 @@ public class DataRegion implements IDataRegionForQuery {
           lastFlushTimeMap.getGlobalFlushedTime(insertTabletNode.getDevicePath().getFullPath());
       startTime = System.nanoTime();
       tryToUpdateBatchInsertLastCache(insertTabletNode, globalLatestFlushedTime);
-      PerformanceOverviewMetricsManager.recordScheduleUpdateLastCacheCost(
-          System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleUpdateLastCacheCost(System.nanoTime() - startTime);
 
       if (!noFailure) {
         throw new BatchProcessException(results);
@@ -1174,8 +1175,7 @@ public class DataRegion implements IDataRegionForQuery {
 
     long startTime = System.nanoTime();
     tryToUpdateInsertLastCache(insertRowNode, globalLatestFlushTime);
-    PerformanceOverviewMetricsManager.recordScheduleUpdateLastCacheCost(
-        System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleUpdateLastCacheCost(System.nanoTime() - startTime);
 
     // check memtable size and may asyncTryToFlush the work memtable
     if (tsFileProcessor.shouldFlush()) {
@@ -3097,7 +3097,7 @@ public class DataRegion implements IDataRegionForQuery {
     }
     long startTime = System.nanoTime();
     writeLock("InsertRowsOfOneDevice");
-    PerformanceOverviewMetricsManager.recordScheduleLockCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleLockCost(System.nanoTime() - startTime);
     try {
       if (deleted) {
         return;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 2d078e4109..3d399ecdff 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.AlignedPath;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -47,7 +48,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
 import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
 import org.apache.iotdb.db.metadata.utils.ResourceByPathUtils;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
@@ -170,6 +170,9 @@ public class TsFileProcessor {
 
   private final QueryMetricsManager queryMetricsManager = QueryMetricsManager.getInstance();
 
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
+
   @SuppressWarnings("squid:S107")
   TsFileProcessor(
       String storageGroupName,
@@ -228,8 +231,7 @@ public class TsFileProcessor {
     if (workMemTable == null) {
       long startTime = System.nanoTime();
       createNewWorkingMemTable();
-      PerformanceOverviewMetricsManager.recordCreateMemtableBlockCost(
-          System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordCreateMemtableBlockCost(System.nanoTime() - startTime);
     }
 
     long[] memIncrements = null;
@@ -246,8 +248,7 @@ public class TsFileProcessor {
                 insertRowNode.getDevicePath().getFullPath(), insertRowNode.getMeasurements(),
                 insertRowNode.getDataTypes(), insertRowNode.getValues());
       }
-      PerformanceOverviewMetricsManager.recordScheduleMemoryBlockCost(
-          System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleMemoryBlockCost(System.nanoTime() - startTime);
     }
 
     long startTime = System.nanoTime();
@@ -266,7 +267,7 @@ public class TsFileProcessor {
               storageGroupName, tsFileResource.getTsFile().getAbsolutePath()),
           e);
     } finally {
-      PerformanceOverviewMetricsManager.recordScheduleWalCost(System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleWalCost(System.nanoTime() - startTime);
     }
 
     startTime = System.nanoTime();
@@ -285,7 +286,7 @@ public class TsFileProcessor {
       tsFileResource.updateEndTime(
           insertRowNode.getDeviceID().toStringID(), insertRowNode.getTime());
     }
-    PerformanceOverviewMetricsManager.recordScheduleMemTableCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleMemTableCost(System.nanoTime() - startTime);
   }
 
   private void createNewWorkingMemTable() throws WriteProcessException {
@@ -310,8 +311,7 @@ public class TsFileProcessor {
     if (workMemTable == null) {
       long startTime = System.nanoTime();
       createNewWorkingMemTable();
-      PerformanceOverviewMetricsManager.recordCreateMemtableBlockCost(
-          System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordCreateMemtableBlockCost(System.nanoTime() - startTime);
     }
 
     long[] memIncrements = null;
@@ -337,8 +337,7 @@ public class TsFileProcessor {
                   start,
                   end);
         }
-        PerformanceOverviewMetricsManager.recordScheduleMemoryBlockCost(
-            System.nanoTime() - startTime);
+        PERFORMANCE_OVERVIEW_METRICS.recordScheduleMemoryBlockCost(System.nanoTime() - startTime);
       }
     } catch (WriteProcessException e) {
       for (int i = start; i < end; i++) {
@@ -363,7 +362,7 @@ public class TsFileProcessor {
       }
       throw new WriteProcessException(e);
     } finally {
-      PerformanceOverviewMetricsManager.recordScheduleWalCost(System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleWalCost(System.nanoTime() - startTime);
     }
 
     startTime = System.nanoTime();
@@ -392,7 +391,7 @@ public class TsFileProcessor {
       tsFileResource.updateEndTime(
           insertTabletNode.getDeviceID().toStringID(), insertTabletNode.getTimes()[end - 1]);
     }
-    PerformanceOverviewMetricsManager.recordScheduleMemTableCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleMemTableCost(System.nanoTime() - startTime);
   }
 
   @SuppressWarnings("squid:S3776") // high Cognitive Complexity
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/executor/RegionWriteExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/executor/RegionWriteExecutor.java
index 1d4d1a7c1c..9c5917664b 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/executor/RegionWriteExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/executor/RegionWriteExecutor.java
@@ -28,6 +28,7 @@ import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.consensus.ConsensusFactory;
 import org.apache.iotdb.consensus.common.response.ConsensusWriteResponse;
 import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -40,7 +41,6 @@ import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
 import org.apache.iotdb.db.metadata.schemaregion.SchemaEngine;
 import org.apache.iotdb.db.metadata.template.ClusterTemplateManager;
 import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.plan.analyze.schema.SchemaValidator;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanVisitor;
@@ -83,6 +83,9 @@ public class RegionWriteExecutor {
 
   private static final DataNodeRegionManager REGION_MANAGER = DataNodeRegionManager.getInstance();
 
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
+
   public RegionExecutionResult execute(ConsensusGroupId groupId, PlanNode planNode) {
     try {
       WritePlanNodeExecutionContext context =
@@ -119,8 +122,7 @@ public class RegionWriteExecutor {
 
       long startWriteTime = System.nanoTime();
       writeResponse = DataRegionConsensusImpl.getInstance().write(groupId, planNode);
-      PerformanceOverviewMetricsManager.recordScheduleStorageCost(
-          System.nanoTime() - startWriteTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleStorageCost(System.nanoTime() - startWriteTime);
 
       // fire Trigger after the insertion
       if (writeResponse.isSuccessful()) {
@@ -135,7 +137,7 @@ public class RegionWriteExecutor {
         triggerCostTime += (System.nanoTime() - startTime);
       }
     }
-    PerformanceOverviewMetricsManager.recordScheduleTriggerCost(triggerCostTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleTriggerCost(triggerCostTime);
     return writeResponse;
   }
 
@@ -239,7 +241,7 @@ public class RegionWriteExecutor {
           }
           return response;
         } finally {
-          PerformanceOverviewMetricsManager.recordScheduleSchemaValidateCost(
+          PERFORMANCE_OVERVIEW_METRICS.recordScheduleSchemaValidateCost(
               System.nanoTime() - startTime);
         }
         boolean hasFailedMeasurement = insertNode.hasFailedMeasurements();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/metric/PerformanceOverviewMetricsManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/metric/PerformanceOverviewMetricsManager.java
deleted file mode 100644
index 1dbab30601..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/metric/PerformanceOverviewMetricsManager.java
+++ /dev/null
@@ -1,214 +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.iotdb.db.mpp.metric;
-
-import org.apache.iotdb.commons.service.metric.MetricService;
-import org.apache.iotdb.commons.service.metric.enums.Metric;
-import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
-import org.apache.iotdb.commons.service.metric.enums.Tag;
-import org.apache.iotdb.metrics.type.Timer;
-import org.apache.iotdb.metrics.utils.MetricLevel;
-
-public class PerformanceOverviewMetricsManager {
-  private static final MetricService metricService = MetricService.getInstance();
-
-  // region overview
-  private static final String PERFORMANCE_OVERVIEW_DETAIL =
-      Metric.PERFORMANCE_OVERVIEW_DETAIL.toString();
-  private static final Timer AUTH_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.AUTHORITY);
-  private static final Timer PARSER_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.PARSER);
-  private static final Timer ANALYZE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.ANALYZER);
-  private static final Timer PLAN_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.PLANNER);
-  private static final Timer SCHEDULE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.SCHEDULER);
-
-  /** Record the time cost in authority stage. */
-  public static void recordAuthCost(long costTimeInNanos) {
-    AUTH_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  /** Record the time cost in parse stage. */
-  public static void recordParseCost(long costTimeInNanos) {
-    PARSER_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordAnalyzeCost(long costTimeInNanos) {
-    ANALYZE_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordPlanCost(long costTimeInNanos) {
-    PLAN_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleCost(long costTimeInNanos) {
-    SCHEDULE_TIMER.updateNanos(costTimeInNanos);
-  }
-  // endregion
-
-  // region schedule
-  private static final String PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL =
-      Metric.PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL.toString();
-  private static final Timer LOCAL_SCHEDULE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.LOCAL_SCHEDULE);
-  private static final Timer REMOTE_SCHEDULE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_SCHEDULE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.REMOTE_SCHEDULE);
-
-  public static void recordScheduleLocalCost(long costTimeInNanos) {
-    LOCAL_SCHEDULE_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleRemoteCost(long costTimeInNanos) {
-    REMOTE_SCHEDULE_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  // endregion
-
-  // region local schedule
-  private static final String PERFORMANCE_OVERVIEW_LOCAL_DETAIL =
-      Metric.PERFORMANCE_OVERVIEW_LOCAL_DETAIL.toString();
-  private static final Timer SCHEMA_VALIDATE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_LOCAL_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.SCHEMA_VALIDATE);
-  private static final Timer TRIGGER_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_LOCAL_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.TRIGGER);
-  private static final Timer STORAGE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_LOCAL_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.STORAGE);
-
-  public static void recordScheduleSchemaValidateCost(long costTimeInNanos) {
-    SCHEMA_VALIDATE_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleTriggerCost(long costTimeInNanos) {
-    TRIGGER_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleStorageCost(long costTimeInNanos) {
-    STORAGE_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  // endregion
-
-  // region engine
-  private static final String PERFORMANCE_OVERVIEW_ENGINE_DETAIL =
-      Metric.PERFORMANCE_OVERVIEW_ENGINE_DETAIL.toString();
-  private static final Timer LOCK_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.LOCK);
-  private static final Timer CREATE_MEMTABLE_BLOCK_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.CREATE_MEMTABLE_BLOCK);
-  private static final Timer MEMORY_BLOCK_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.MEMORY_BLOCK);
-  private static final Timer WAL_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.WAL);
-  private static final Timer MEMTABLE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.MEMTABLE);
-  private static final Timer LAST_CACHE_TIMER =
-      metricService.getOrCreateTimer(
-          PERFORMANCE_OVERVIEW_ENGINE_DETAIL,
-          MetricLevel.IMPORTANT,
-          Tag.STAGE.toString(),
-          PerformanceOverviewMetrics.LAST_CACHE);
-
-  public static void recordScheduleLockCost(long costTimeInNanos) {
-    LOCK_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordCreateMemtableBlockCost(long costTimeInNanos) {
-    CREATE_MEMTABLE_BLOCK_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleMemoryBlockCost(long costTimeInNanos) {
-    MEMORY_BLOCK_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleWalCost(long costTimeInNanos) {
-    WAL_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleMemTableCost(long costTimeInNanos) {
-    MEMTABLE_TIMER.updateNanos(costTimeInNanos);
-  }
-
-  public static void recordScheduleUpdateLastCacheCost(long costTimeInNanos) {
-    LAST_CACHE_TIMER.updateNanos(costTimeInNanos);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java
index de474a5db2..4c2d0c23bf 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/QueryExecution.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient;
 import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient;
 import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.query.KilledByOthersException;
@@ -34,7 +35,6 @@ import org.apache.iotdb.db.mpp.execution.QueryState;
 import org.apache.iotdb.db.mpp.execution.QueryStateMachine;
 import org.apache.iotdb.db.mpp.execution.exchange.MPPDataExchangeService;
 import org.apache.iotdb.db.mpp.execution.exchange.source.ISourceHandle;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
 import org.apache.iotdb.db.mpp.plan.analyze.Analysis;
 import org.apache.iotdb.db.mpp.plan.analyze.Analyzer;
@@ -134,6 +134,9 @@ public class QueryExecution implements IQueryExecution {
 
   private static final QueryMetricsManager QUERY_METRICS = QueryMetricsManager.getInstance();
 
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
+
   public QueryExecution(
       Statement statement,
       MPPQueryContext context,
@@ -211,7 +214,7 @@ public class QueryExecution implements IQueryExecution {
     if (context.getQueryType() == QueryType.READ) {
       initResultHandle();
     }
-    PerformanceOverviewMetricsManager.recordPlanCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordPlanCost(System.nanoTime() - startTime);
     schedule();
   }
 
@@ -281,7 +284,7 @@ public class QueryExecution implements IQueryExecution {
     try {
       result = new Analyzer(context, partitionFetcher, schemaFetcher).analyze(statement);
     } finally {
-      PerformanceOverviewMetricsManager.recordAnalyzeCost(System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordAnalyzeCost(System.nanoTime() - startTime);
     }
     return result;
   }
@@ -309,7 +312,7 @@ public class QueryExecution implements IQueryExecution {
             syncInternalServiceClientManager,
             asyncInternalServiceClientManager);
     this.scheduler.start();
-    PerformanceOverviewMetricsManager.recordScheduleCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleCost(System.nanoTime() - startTime);
   }
 
   // Use LogicalPlanner to do the logical query plan and logical optimization
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
index d848a9a45e..ccd650336d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/StatementGenerator.java
@@ -23,11 +23,11 @@ import org.apache.iotdb.common.rpc.thrift.TAggregationType;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.db.constant.SqlConstant;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.template.TemplateQueryType;
 import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.plan.expression.binary.GreaterEqualExpression;
 import org.apache.iotdb.db.mpp.plan.expression.binary.LessThanExpression;
 import org.apache.iotdb.db.mpp.plan.expression.binary.LogicAndExpression;
@@ -108,7 +108,9 @@ import java.util.Map;
 
 /** Convert SQL and RPC requests to {@link Statement}. */
 public class StatementGenerator {
-  // TODO @spricoder optimize the method adding metrics
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
+
   public static Statement createStatement(String sql, ZoneId zoneId) {
     return invokeParser(sql, zoneId);
   }
@@ -146,7 +148,7 @@ public class StatementGenerator {
     queryStatement.setSelectComponent(selectComponent);
     queryStatement.setFromComponent(fromComponent);
     queryStatement.setWhereCondition(whereCondition);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return queryStatement;
   }
 
@@ -180,7 +182,7 @@ public class StatementGenerator {
     lastQueryStatement.setSelectComponent(selectComponent);
     lastQueryStatement.setFromComponent(fromComponent);
     lastQueryStatement.setWhereCondition(whereCondition);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
 
     return lastQueryStatement;
   }
@@ -236,7 +238,7 @@ public class StatementGenerator {
       whereCondition.setPredicate(predicate);
       queryStatement.setWhereCondition(whereCondition);
     }
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return queryStatement;
   }
 
@@ -250,7 +252,7 @@ public class StatementGenerator {
     insertStatement.setMeasurements(insertRecordReq.getMeasurements().toArray(new String[0]));
     insertStatement.setAligned(insertRecordReq.isAligned);
     insertStatement.fillValues(insertRecordReq.values);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -266,7 +268,7 @@ public class StatementGenerator {
     insertStatement.setValues(insertRecordReq.getValues().toArray(new Object[0]));
     insertStatement.setNeedInferType(true);
     insertStatement.setAligned(insertRecordReq.isAligned);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -295,7 +297,7 @@ public class StatementGenerator {
     }
     insertStatement.setDataTypes(dataTypes);
     insertStatement.setAligned(insertTabletReq.isAligned);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -334,7 +336,7 @@ public class StatementGenerator {
       insertTabletStatementList.add(insertTabletStatement);
     }
     insertStatement.setInsertTabletStatementList(insertTabletStatementList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -358,7 +360,7 @@ public class StatementGenerator {
       insertRowStatementList.add(statement);
     }
     insertStatement.setInsertRowStatementList(insertRowStatementList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -384,7 +386,7 @@ public class StatementGenerator {
       insertRowStatementList.add(statement);
     }
     insertStatement.setInsertRowStatementList(insertRowStatementList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -409,7 +411,7 @@ public class StatementGenerator {
       insertRowStatementList.add(statement);
     }
     insertStatement.setInsertRowStatementList(insertRowStatementList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -436,7 +438,7 @@ public class StatementGenerator {
       insertRowStatementList.add(statement);
     }
     insertStatement.setInsertRowStatementList(insertRowStatementList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return insertStatement;
   }
 
@@ -447,7 +449,7 @@ public class StatementGenerator {
     DatabaseSchemaStatement statement =
         new DatabaseSchemaStatement(DatabaseSchemaStatement.DatabaseSchemaStatementType.CREATE);
     statement.setStorageGroupPath(parseStorageGroupRawString(storageGroup));
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -464,7 +466,7 @@ public class StatementGenerator {
     statement.setTags(req.tags);
     statement.setAttributes(req.attributes);
     statement.setAlias(req.measurementAlias);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -493,7 +495,7 @@ public class StatementGenerator {
     statement.setTagsList(req.tagsList);
     statement.setAttributesList(req.attributesList);
     statement.setAliasList(req.measurementAlias);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -526,7 +528,7 @@ public class StatementGenerator {
     statement.setTagsList(req.tagsList);
     statement.setAttributesList(req.attributesList);
     statement.setAliasList(req.measurementAliasList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -538,7 +540,7 @@ public class StatementGenerator {
       parseStorageGroupRawString(path);
     }
     statement.setPrefixPath(storageGroups);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -553,7 +555,7 @@ public class StatementGenerator {
     statement.setPathList(pathList);
     statement.setDeleteStartTime(req.getStartTime());
     statement.setDeleteEndTime(req.getEndTime());
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -644,7 +646,7 @@ public class StatementGenerator {
     CreateSchemaTemplateStatement statement =
         new CreateSchemaTemplateStatement(
             req.getName(), measurements, dataTypes, encodings, compressors, alignedPrefix.keySet());
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -669,7 +671,7 @@ public class StatementGenerator {
       default:
         break;
     }
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return result;
   }
 
@@ -678,7 +680,7 @@ public class StatementGenerator {
     long startTime = System.nanoTime();
     SetSchemaTemplateStatement statement =
         new SetSchemaTemplateStatement(req.getTemplateName(), new PartialPath(req.getPrefixPath()));
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -688,14 +690,14 @@ public class StatementGenerator {
     UnsetSchemaTemplateStatement statement =
         new UnsetSchemaTemplateStatement(
             req.getTemplateName(), new PartialPath(req.getPrefixPath()));
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
   public static DropSchemaTemplateStatement createStatement(TSDropSchemaTemplateReq req) {
     final long startTime = System.nanoTime();
     DropSchemaTemplateStatement statement = new DropSchemaTemplateStatement(req.getTemplateName());
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -707,7 +709,7 @@ public class StatementGenerator {
       pathPatternList.add(new PartialPath(pathPatternString));
     }
     DeleteTimeSeriesStatement statement = new DeleteTimeSeriesStatement(pathPatternList);
-    PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+    PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     return statement;
   }
 
@@ -757,7 +759,7 @@ public class StatementGenerator {
       }
       return astVisitor.visit(tree);
     } finally {
-      PerformanceOverviewMetricsManager.recordParseCost(System.nanoTime() - startTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordParseCost(System.nanoTime() - startTime);
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/AsyncSendPlanNodeHandler.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/AsyncSendPlanNodeHandler.java
index d0cb0e9679..2bd50a6c7e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/AsyncSendPlanNodeHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/AsyncSendPlanNodeHandler.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.plan.scheduler;
 
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.mpp.rpc.thrift.TSendPlanNodeResp;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -33,6 +33,8 @@ public class AsyncSendPlanNodeHandler implements AsyncMethodCallback<TSendPlanNo
   private final AtomicLong pendingNumber;
   private final Map<Integer, TSendPlanNodeResp> instanceId2RespMap;
   private final long sendTime;
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
 
   public AsyncSendPlanNodeHandler(
       int instanceId,
@@ -49,7 +51,7 @@ public class AsyncSendPlanNodeHandler implements AsyncMethodCallback<TSendPlanNo
   public void onComplete(TSendPlanNodeResp tSendPlanNodeResp) {
     instanceId2RespMap.put(instanceId, tSendPlanNodeResp);
     if (pendingNumber.decrementAndGet() == 0) {
-      PerformanceOverviewMetricsManager.recordScheduleRemoteCost(System.nanoTime() - sendTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleRemoteCost(System.nanoTime() - sendTime);
       synchronized (pendingNumber) {
         pendingNumber.notifyAll();
       }
@@ -66,7 +68,7 @@ public class AsyncSendPlanNodeHandler implements AsyncMethodCallback<TSendPlanNo
         RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode(), errorMsg));
     instanceId2RespMap.put(instanceId, resp);
     if (pendingNumber.decrementAndGet() == 0) {
-      PerformanceOverviewMetricsManager.recordScheduleRemoteCost(System.nanoTime() - sendTime);
+      PERFORMANCE_OVERVIEW_METRICS.recordScheduleRemoteCost(System.nanoTime() - sendTime);
       synchronized (pendingNumber) {
         pendingNumber.notifyAll();
       }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/FragmentInstanceDispatcherImpl.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/FragmentInstanceDispatcherImpl.java
index 0719b86452..b990e0e4fb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/FragmentInstanceDispatcherImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/scheduler/FragmentInstanceDispatcherImpl.java
@@ -26,13 +26,13 @@ import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.mpp.FragmentInstanceDispatchException;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.execution.executor.RegionExecutionResult;
 import org.apache.iotdb.db.mpp.execution.executor.RegionReadExecutor;
 import org.apache.iotdb.db.mpp.execution.executor.RegionWriteExecutor;
-import org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager;
 import org.apache.iotdb.db.mpp.metric.QueryMetricsManager;
 import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
 import org.apache.iotdb.db.mpp.plan.planner.plan.FragmentInstance;
@@ -75,6 +75,8 @@ public class FragmentInstanceDispatcherImpl implements IFragInstanceDispatcher {
       asyncInternalServiceClientManager;
 
   private static final QueryMetricsManager QUERY_METRICS = QueryMetricsManager.getInstance();
+  private static final PerformanceOverviewMetrics PERFORMANCE_OVERVIEW_METRICS =
+      PerformanceOverviewMetrics.getInstance();
 
   public FragmentInstanceDispatcherImpl(
       QueryType type,
@@ -194,7 +196,7 @@ public class FragmentInstanceDispatcherImpl implements IFragInstanceDispatcher {
                 TSStatusCode.INTERNAL_SERVER_ERROR, "Unexpected errors: " + t.getMessage()));
       }
     }
-    PerformanceOverviewMetricsManager.recordScheduleLocalCost(
+    PERFORMANCE_OVERVIEW_METRICS.recordScheduleLocalCost(
         System.nanoTime() - localScheduleStartTime);
     // wait until remote dispatch done
     try {
diff --git a/server/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java b/server/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java
index 453f4f9933..664b0c3a67 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/metrics/DataNodeMetricsHelper.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.service.metrics;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.service.metric.MetricService;
+import org.apache.iotdb.commons.service.metric.enums.PerformanceOverviewMetrics;
 import org.apache.iotdb.db.mpp.metric.DataExchangeCostMetricSet;
 import org.apache.iotdb.db.mpp.metric.DataExchangeCountMetricSet;
 import org.apache.iotdb.db.mpp.metric.DriverSchedulerMetricSet;
@@ -51,5 +52,8 @@ public class DataNodeMetricsHelper {
     MetricService.getInstance().addMetricSet(new DataExchangeCostMetricSet());
     MetricService.getInstance().addMetricSet(new DataExchangeCountMetricSet());
     MetricService.getInstance().addMetricSet(new DriverSchedulerMetricSet());
+
+    // bind performance overview related metrics
+    MetricService.getInstance().addMetricSet(PerformanceOverviewMetrics.getInstance());
   }
 }


[iotdb] 03/04: Seperate disk dashboard (#9413)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 0e5bd28b9cd600a82bfc68984607670625f534e2
Author: Liu Xuxin <37...@users.noreply.github.com>
AuthorDate: Wed Mar 22 22:53:54 2023 +0800

    Seperate disk dashboard (#9413)
---
 .../Apache-IoTDB-ConfigNode-Dashboard.json         | 2586 ++++++++------------
 .../Apache-IoTDB-DataNode-Dashboard.json           | 1147 ++-------
 .../Apache-IoTDB-Disk-IO-Dashboard.json            |  894 +++++++
 .../iotdb/metrics/metricsets/disk/DiskMetrics.java |   56 +-
 4 files changed, 2071 insertions(+), 2612 deletions(-)

diff --git a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
index 7016071f59..24a9f1a04f 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
@@ -9,7 +9,7 @@
       "pluginName": "Prometheus"
     }
   ],
-  "__elements": [],
+  "__elements": {},
   "__requires": [
     {
       "type": "panel",
@@ -21,7 +21,7 @@
       "type": "grafana",
       "id": "grafana",
       "name": "Grafana",
-      "version": "8.4.2"
+      "version": "9.3.6"
     },
     {
       "type": "panel",
@@ -75,12 +75,15 @@
   "fiscalYearStartMonth": 0,
   "graphTooltip": 1,
   "id": null,
-  "iteration": 1678983654736,
   "links": [],
   "liveNow": false,
   "panels": [
     {
       "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -90,6 +93,10 @@
       "id": 57,
       "panels": [
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The current status of cluster ConfigNodes",
           "fieldConfig": {
             "defaults": {
@@ -276,6 +283,10 @@
           "type": "piechart"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of cluster Databases",
           "fieldConfig": {
             "defaults": {
@@ -287,8 +298,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -336,6 +346,10 @@
           "type": "stat"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of cluster DataRegions",
           "fieldConfig": {
             "defaults": {
@@ -347,8 +361,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -396,6 +409,10 @@
           "type": "stat"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of cluster SchemaRegions",
           "fieldConfig": {
             "defaults": {
@@ -407,8 +424,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -471,8 +487,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -522,6 +537,10 @@
           "type": "stat"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The total disk space of current ConfigNode",
           "fieldConfig": {
             "defaults": {
@@ -533,8 +552,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -583,6 +601,10 @@
           "type": "stat"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The current status of cluster DataNodes",
           "fieldConfig": {
             "defaults": {
@@ -709,6 +731,10 @@
           "type": "piechart"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "Bar of current ConfigNode",
           "fieldConfig": {
             "defaults": {
@@ -722,8 +748,7 @@
                 "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   },
                   {
                     "color": "#EAB839",
@@ -820,6 +845,10 @@
           "type": "bargauge"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The current status of cluster DataRegions",
           "fieldConfig": {
             "defaults": {
@@ -947,6 +976,10 @@
           "type": "piechart"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The current status of cluster SchemaRegions",
           "fieldConfig": {
             "defaults": {
@@ -1088,8 +1121,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -1151,8 +1183,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -1200,11 +1231,24 @@
           "type": "stat"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "Overview",
       "type": "row"
     },
     {
       "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -1214,6 +1258,10 @@
       "id": 49,
       "panels": [
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of cluster ConfigNodes",
           "fieldConfig": {
             "defaults": {
@@ -1273,7 +1321,8 @@
                 "lastNotNull"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "single",
@@ -1300,6 +1349,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The status history of cluster ConfigNodes",
           "fieldConfig": {
             "defaults": {
@@ -1447,6 +1500,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The status history of cluster DataNodes",
           "fieldConfig": {
             "defaults": {
@@ -1594,6 +1651,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of SchemaRegions in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -1680,6 +1741,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of SchemaRegionGroup leadere in each DataNodes",
           "fieldConfig": {
             "defaults": {
@@ -1766,6 +1831,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of DataRegions in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -1852,6 +1921,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of DataRegionGroup leaders in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -1938,11 +2011,24 @@
           "type": "timeseries"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "NodeInfo",
       "type": "row"
     },
     {
       "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -1952,6 +2038,10 @@
       "id": 53,
       "panels": [
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of SchemaRegionGroups in each Database",
           "fieldConfig": {
             "defaults": {
@@ -2038,6 +2128,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of DataRegionGroups in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -2124,6 +2218,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The number of SeriesSlots in each Database",
           "fieldConfig": {
             "defaults": {
@@ -2210,11 +2308,24 @@
           "type": "timeseries"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "Database",
       "type": "row"
     },
     {
       "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -2224,6 +2335,10 @@
       "id": 81,
       "panels": [
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The status history of cluster DataRegions",
           "fieldConfig": {
             "defaults": {
@@ -2371,6 +2486,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "description": "The status history of cluster SchemaRegions",
           "fieldConfig": {
             "defaults": {
@@ -2522,11 +2641,24 @@
           "type": "timeseries"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "Region",
       "type": "row"
     },
     {
-      "collapsed": false,
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -2534,1124 +2666,129 @@
         "y": 4
       },
       "id": 109,
-      "panels": [],
-      "title": "ConfigRegion Ratis Consensus",
-      "type": "row"
-    },
-    {
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "axisLabel": "",
-            "axisPlacement": "auto",
-            "barAlignment": 0,
-            "drawStyle": "line",
-            "fillOpacity": 10,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
+      "panels": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 10,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "never",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [
+                {
+                  "options": {
+                    "match": "empty",
+                    "result": {
+                      "index": 0,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                },
+                {
+                  "options": {
+                    "match": "null+nan",
+                    "result": {
+                      "index": 1,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                }
+              ],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              },
+              "unit": "s"
             },
-            "showPoints": "never",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 9,
+            "w": 12,
+            "x": 0,
+            "y": 5
+          },
+          "id": 111,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "desc"
             }
           },
-          "mappings": [
+          "targets": [
             {
-              "options": {
-                "match": "empty",
-                "result": {
-                  "index": 0,
-                  "text": "0"
-                }
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
               },
-              "type": "special"
+              "exemplar": true,
+              "expr": "rate(ConfigRegion_ratis_log_worker_appendEntryLatency_seconds_sum{instance=\"$instance\"}[1m])/rate(ConfigRegion_ratis_log_worker_appendEntryLatency_seconds_count{instance=\"$instance\"}[1m])",
+              "interval": "",
+              "legendFormat": "appendEntryLatency",
+              "refId": "A"
             },
             {
-              "options": {
-                "match": "null+nan",
-                "result": {
-                  "index": 1,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            }
-          ],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
               },
-              {
-                "color": "red",
-                "value": 80
-              }
-            ]
-          },
-          "unit": "s"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 9,
-        "w": 12,
-        "x": 0,
-        "y": 5
-      },
-      "id": 111,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
-          ],
-          "displayMode": "table",
-          "placement": "right"
-        },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "desc"
-        }
-      },
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_log_worker_appendEntryLatency_seconds_sum{instance=\"$instance\"}[1m])/rate(ConfigRegion_ratis_log_worker_appendEntryLatency_seconds_count{instance=\"$instance\"}[1m])",
-          "interval": "",
-          "legendFormat": "appendEntryLatency",
-          "refId": "A"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_server_follower_append_entry_latency_seconds_sum{instance=\"$instance\"}[1m])/rate(ConfigRegion_ratis_server_follower_append_entry_latency_seconds_count{instance=\"$instance\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "followerAppendEntryLatency",
-          "refId": "B"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeStateMachine\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeStateMachine\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "writeStateMachine",
-          "refId": "C"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_server_clientWriteRequest_seconds_sum{instance=\"$instance\"}[1m])/rate(ConfigRegion_ratis_server_clientWriteRequest_seconds_count{instance=\"$instance\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "clientWriteRequest",
-          "refId": "D"
-        }
-      ],
-      "title": "Ratis Stage Time",
-      "type": "timeseries"
-    },
-    {
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "axisLabel": "",
-            "axisPlacement": "auto",
-            "barAlignment": 0,
-            "drawStyle": "line",
-            "fillOpacity": 10,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
-            },
-            "showPoints": "never",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
-            },
-            "thresholdsStyle": {
-              "mode": "off"
-            }
-          },
-          "mappings": [
-            {
-              "options": {
-                "match": "empty",
-                "result": {
-                  "index": 0,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            },
-            {
-              "options": {
-                "match": "null+nan",
-                "result": {
-                  "index": 1,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            }
-          ],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              },
-              {
-                "color": "red",
-                "value": 80
-              }
-            ]
-          },
-          "unit": "s"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 9,
-        "w": 12,
-        "x": 12,
-        "y": 5
-      },
-      "id": 112,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
-          ],
-          "displayMode": "table",
-          "placement": "right"
-        },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "desc"
-        }
-      },
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_log_worker_queueingDelay_seconds_sum{instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_queueingDelay_seconds_count{instance=\"$instance\"}[1m])",
-          "interval": "",
-          "legendFormat": "queueingDelay",
-          "refId": "A"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_log_worker_enqueuedTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_enqueuedTime_seconds_count{instance=\"$instance\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "enqueuedTime",
-          "refId": "B"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_log_worker_writelogExecutionTime_seconds_sum {instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_writelogExecutionTime_seconds_count{instance=\"$instance\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "writelogExecutionTime",
-          "refId": "C"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_log_worker_flushTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_flushTime_seconds_count{instance=\"$instance\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "flushTime",
-          "refId": "D"
-        }
-      ],
-      "title": "Write Log Entry",
-      "type": "timeseries"
-    },
-    {
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "axisLabel": "",
-            "axisPlacement": "auto",
-            "barAlignment": 0,
-            "drawStyle": "line",
-            "fillOpacity": 10,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
-            },
-            "showPoints": "never",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
-            },
-            "thresholdsStyle": {
-              "mode": "off"
-            }
-          },
-          "mappings": [
-            {
-              "options": {
-                "match": "empty",
-                "result": {
-                  "index": 0,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            },
-            {
-              "options": {
-                "match": "null+nan",
-                "result": {
-                  "index": 1,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            }
-          ],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              },
-              {
-                "color": "red",
-                "value": 80
-              }
-            ]
-          },
-          "unit": "s"
-        },
-        "overrides": [
-          {
-            "__systemRef": "hideSeriesFrom",
-            "matcher": {
-              "id": "byNames",
-              "options": {
-                "mode": "exclude",
-                "names": [
-                  "writeLocally"
-                ],
-                "prefix": "All except:",
-                "readOnly": true
-              }
-            },
-            "properties": [
-              {
-                "id": "custom.hideFrom",
-                "value": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": true
-                }
-              }
-            ]
-          }
-        ]
-      },
-      "gridPos": {
-        "h": 9,
-        "w": 8,
-        "x": 0,
-        "y": 14
-      },
-      "id": 113,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
-          ],
-          "displayMode": "table",
-          "placement": "right"
-        },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "desc"
-        }
-      },
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeLocally\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
-          "interval": "",
-          "legendFormat": "{{stage}}",
-          "refId": "A"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeRemotely\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "{{stage}}",
-          "refId": "B"
-        }
-      ],
-      "title": "Remote / Local Write Time",
-      "type": "timeseries"
-    },
-    {
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "axisLabel": "",
-            "axisPlacement": "auto",
-            "barAlignment": 0,
-            "drawStyle": "line",
-            "fillOpacity": 10,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
-            },
-            "showPoints": "never",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
-            },
-            "thresholdsStyle": {
-              "mode": "off"
-            }
-          },
-          "mappings": [
-            {
-              "options": {
-                "match": "empty",
-                "result": {
-                  "index": 0,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            },
-            {
-              "options": {
-                "match": "null+nan",
-                "result": {
-                  "index": 1,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            }
-          ],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              },
-              {
-                "color": "red",
-                "value": 80
-              }
-            ]
-          },
-          "unit": "reqps"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 9,
-        "w": 8,
-        "x": 8,
-        "y": 14
-      },
-      "id": 114,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
-          ],
-          "displayMode": "table",
-          "placement": "right"
-        },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "desc"
-        }
-      },
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
-          "interval": "",
-          "legendFormat": "{{stage}}",
-          "refId": "A"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "{{stage}}",
-          "refId": "B"
-        }
-      ],
-      "title": "Remote / Local Write QPS",
-      "type": "timeseries"
-    },
-    {
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "axisLabel": "",
-            "axisPlacement": "auto",
-            "barAlignment": 0,
-            "drawStyle": "line",
-            "fillOpacity": 10,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
-            },
-            "showPoints": "never",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
-            },
-            "thresholdsStyle": {
-              "mode": "off"
-            }
-          },
-          "mappings": [
-            {
-              "options": {
-                "match": "empty",
-                "result": {
-                  "index": 0,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            },
-            {
-              "options": {
-                "match": "null+nan",
-                "result": {
-                  "index": 1,
-                  "text": "0"
-                }
-              },
-              "type": "special"
-            }
-          ],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              },
-              {
-                "color": "red",
-                "value": 80
-              }
-            ]
-          },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 9,
-        "w": 8,
-        "x": 16,
-        "y": 14
-      },
-      "id": 115,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
-          ],
-          "displayMode": "table",
-          "placement": "right"
-        },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "desc"
-        }
-      },
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "ConfigRegion_ratis_log_worker_closedSegmentsSizeInBytes {instance=\"$instance\"}",
-          "interval": "",
-          "legendFormat": "closedSegmentsSizeInBytes",
-          "refId": "A"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "exemplar": true,
-          "expr": "ConfigRegion_ratis_log_worker_openSegmentSizeInBytes {instance=\"$instance\"}",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "openSegmentSizeInBytes",
-          "refId": "B"
-        }
-      ],
-      "title": "RatisConsensus Memory",
-      "type": "timeseries"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 23
-      },
-      "id": 13,
-      "panels": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "axisSoftMin": 0,
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 10,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "percent"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 5
-          },
-          "id": 15,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
-              "interval": "",
-              "legendFormat": "System CPU Load",
-              "range": true,
-              "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
-              "hide": false,
-              "legendFormat": "Process CPU Load",
-              "range": true,
-              "refId": "B"
-            }
-          ],
-          "title": "CPU Load",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "axisSoftMin": 0,
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 10,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "ns"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 5
-          },
-          "id": 16,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
-              "hide": false,
-              "legendFormat": "Process CPU Time",
-              "range": true,
-              "refId": "B"
-            }
-          ],
-          "title": "CPU Time Per Minute",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 10,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "none"
-            },
-            "overrides": [
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "warn"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "orange",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
-              },
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "trace"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "purple",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
-              },
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "error"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "red",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
-              }
-            ]
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 5
-          },
-          "id": 25,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "{{level}}",
-              "range": true,
-              "refId": "C"
-            }
-          ],
-          "title": "Log Number Per Minute",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "bytes"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 13
-          },
-          "id": 17,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}-sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"}",
-              "interval": "",
-              "legendFormat": "Used physical memory",
-              "range": true,
-              "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+              "expr": "rate(ConfigRegion_ratis_server_follower_append_entry_latency_seconds_sum{instance=\"$instance\"}[1m])/rate(ConfigRegion_ratis_server_follower_append_entry_latency_seconds_count{instance=\"$instance\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "Total physical memory",
-              "range": true,
+              "legendFormat": "followerAppendEntryLatency",
               "refId": "B"
             },
             {
@@ -3659,147 +2796,27 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "sys_committed_vm_size{instance=\"$instance\"}",
+              "exemplar": true,
+              "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeStateMachine\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeStateMachine\"}[1m])",
               "hide": false,
-              "legendFormat": "Committed vm size",
-              "range": true,
+              "interval": "",
+              "legendFormat": "writeStateMachine",
               "refId": "C"
-            }
-          ],
-          "title": "System Memory",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "bytes"
-            },
-            "overrides": [
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Total Swap Size"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "green",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
-              },
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Used Swap Size"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "yellow",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
-              }
-            ]
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 13
-          },
-          "id": 18,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
             },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
-            }
-          },
-          "targets": [
             {
               "datasource": {
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"} - sys_free_swap_space_size{instance=\"$instance\", name=\"system\"}",
+              "expr": "rate(ConfigRegion_ratis_server_clientWriteRequest_seconds_sum{instance=\"$instance\"}[1m])/rate(ConfigRegion_ratis_server_clientWriteRequest_seconds_count{instance=\"$instance\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "Used Swap Size",
-              "range": true,
-              "refId": "B"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
-              "hide": false,
-              "legendFormat": "Total Swap Size",
-              "range": true,
-              "refId": "C"
+              "legendFormat": "clientWriteRequest",
+              "refId": "D"
             }
           ],
-          "title": "System Swap Size",
+          "title": "Ratis Stage Time",
           "type": "timeseries"
         },
         {
@@ -3813,11 +2830,13 @@
                 "mode": "palette-classic"
               },
               "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -3840,12 +2859,34 @@
                   "mode": "off"
                 }
               },
-              "mappings": [],
+              "mappings": [
+                {
+                  "options": {
+                    "match": "empty",
+                    "result": {
+                      "index": 0,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                },
+                {
+                  "options": {
+                    "match": "null+nan",
+                    "result": {
+                      "index": 1,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                }
+              ],
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green"
+                    "color": "green",
+                    "value": null
                   },
                   {
                     "color": "red",
@@ -3853,22 +2894,24 @@
                   }
                 ]
               },
-              "unit": "bytes"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 13
+            "h": 9,
+            "w": 12,
+            "x": 12,
+            "y": 5
           },
-          "id": 19,
+          "id": 112,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
               "showLegend": true
             },
             "tooltip": {
@@ -3882,10 +2925,10 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "process_max_mem{instance=~\"${instance}\"}",
-              "legendFormat": "Max Memory",
-              "range": true,
+              "exemplar": true,
+              "expr": "rate(ConfigRegion_ratis_log_worker_queueingDelay_seconds_sum{instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_queueingDelay_seconds_count{instance=\"$instance\"}[1m])",
+              "interval": "",
+              "legendFormat": "queueingDelay",
               "refId": "A"
             },
             {
@@ -3893,11 +2936,11 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "process_total_mem{instance=~\"${instance}\"}",
+              "exemplar": true,
+              "expr": "rate(ConfigRegion_ratis_log_worker_enqueuedTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_enqueuedTime_seconds_count{instance=\"$instance\"}[1m])",
               "hide": false,
-              "legendFormat": "Total Memory",
-              "range": true,
+              "interval": "",
+              "legendFormat": "enqueuedTime",
               "refId": "B"
             },
             {
@@ -3905,15 +2948,27 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "process_total_mem{instance=~\"${instance}\"} - process_free_mem{instance=~\"${instance}\"}",
+              "exemplar": true,
+              "expr": "rate(ConfigRegion_ratis_log_worker_writelogExecutionTime_seconds_sum {instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_writelogExecutionTime_seconds_count{instance=\"$instance\"}[1m])",
               "hide": false,
-              "legendFormat": "Used Memory",
-              "range": true,
+              "interval": "",
+              "legendFormat": "writelogExecutionTime",
               "refId": "C"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "rate(ConfigRegion_ratis_log_worker_flushTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(ConfigRegion_ratis_log_worker_flushTime_seconds_count{instance=\"$instance\"}[1m])",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "flushTime",
+              "refId": "D"
             }
           ],
-          "title": "Process Memory",
+          "title": "Write Log Entry",
           "type": "timeseries"
         },
         {
@@ -3927,11 +2982,13 @@
                 "mode": "palette-classic"
               },
               "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -3954,30 +3011,84 @@
                   "mode": "off"
                 }
               },
-              "mappings": [],
+              "mappings": [
+                {
+                  "options": {
+                    "match": "empty",
+                    "result": {
+                      "index": 0,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                },
+                {
+                  "options": {
+                    "match": "null+nan",
+                    "result": {
+                      "index": 1,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                }
+              ],
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green"
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              },
+              "unit": "s"
+            },
+            "overrides": [
+              {
+                "__systemRef": "hideSeriesFrom",
+                "matcher": {
+                  "id": "byNames",
+                  "options": {
+                    "mode": "exclude",
+                    "names": [
+                      "writeLocally"
+                    ],
+                    "prefix": "All except:",
+                    "readOnly": true
+                  }
+                },
+                "properties": [
+                  {
+                    "id": "custom.hideFrom",
+                    "value": {
+                      "legend": false,
+                      "tooltip": false,
+                      "viz": true
+                    }
                   }
                 ]
               }
-            },
-            "overrides": []
+            ]
           },
           "gridPos": {
-            "h": 8,
+            "h": 9,
             "w": 8,
             "x": 0,
-            "y": 21
+            "y": 14
           },
-          "id": 20,
+          "id": 113,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
               "showLegend": true
             },
             "tooltip": {
@@ -3991,12 +3102,10 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
+              "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeLocally\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
               "interval": "",
-              "legendFormat": "Young GC number",
-              "range": true,
+              "legendFormat": "{{stage}}",
               "refId": "A"
             },
             {
@@ -4005,14 +3114,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+              "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeRemotely\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "Full GC number",
+              "legendFormat": "{{stage}}",
               "refId": "B"
             }
           ],
-          "title": "The Number Of GC Per Minute",
+          "title": "Remote / Local Write Time",
           "type": "timeseries"
         },
         {
@@ -4026,6 +3135,8 @@
                 "mode": "palette-classic"
               },
               "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4053,31 +3164,59 @@
                   "mode": "off"
                 }
               },
-              "mappings": [],
+              "mappings": [
+                {
+                  "options": {
+                    "match": "empty",
+                    "result": {
+                      "index": 0,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                },
+                {
+                  "options": {
+                    "match": "null+nan",
+                    "result": {
+                      "index": 1,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                }
+              ],
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green"
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "reqps"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
+            "h": 9,
             "w": 8,
             "x": 8,
-            "y": 21
+            "y": 14
           },
-          "id": 21,
+          "id": 114,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
               "showLegend": true
             },
             "tooltip": {
@@ -4092,9 +3231,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
+              "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
               "interval": "",
-              "legendFormat": "The time of young GC",
+              "legendFormat": "{{stage}}",
               "refId": "A"
             },
             {
@@ -4103,14 +3242,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+              "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "The time of full GC",
+              "legendFormat": "{{stage}}",
               "refId": "B"
             }
           ],
-          "title": "The Time Consumed Of GC Per Minute",
+          "title": "Remote / Local Write QPS",
           "type": "timeseries"
         },
         {
@@ -4124,11 +3263,13 @@
                 "mode": "palette-classic"
               },
               "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -4151,30 +3292,59 @@
                   "mode": "off"
                 }
               },
-              "mappings": [],
+              "mappings": [
+                {
+                  "options": {
+                    "match": "empty",
+                    "result": {
+                      "index": 0,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                },
+                {
+                  "options": {
+                    "match": "null+nan",
+                    "result": {
+                      "index": 1,
+                      "text": "0"
+                    }
+                  },
+                  "type": "special"
+                }
+              ],
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green"
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
                   }
                 ]
-              }
+              },
+              "unit": "bytes"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
+            "h": 9,
             "w": 8,
             "x": 16,
-            "y": 21
+            "y": 14
           },
-          "id": 22,
+          "id": 115,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
               "showLegend": true
             },
             "tooltip": {
@@ -4188,12 +3358,10 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "jvm_threads_live_threads{instance=~\"$instance\"}",
+              "expr": "ConfigRegion_ratis_log_worker_closedSegmentsSizeInBytes {instance=\"$instance\"}",
               "interval": "",
-              "legendFormat": "Total Number",
-              "range": true,
+              "legendFormat": "closedSegmentsSizeInBytes",
               "refId": "A"
             },
             {
@@ -4202,16 +3370,43 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "jvm_threads_states_threads{instance=~\"$instance\"}",
+              "expr": "ConfigRegion_ratis_log_worker_openSegmentSizeInBytes {instance=\"$instance\"}",
               "hide": false,
               "interval": "",
-              "legendFormat": "{{state}}",
+              "legendFormat": "openSegmentSizeInBytes",
               "refId": "B"
             }
           ],
-          "title": "The Number Of Java Thread",
+          "title": "RatisConsensus Memory",
           "type": "timeseries"
-        },
+        }
+      ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
+      "title": "ConfigRegion Ratis Consensus",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 5
+      },
+      "id": 13,
+      "panels": [
         {
           "datasource": {
             "type": "prometheus",
@@ -4225,6 +3420,8 @@
               "custom": {
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 10,
@@ -4252,14 +3449,14 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
                   }
                 ]
               },
-              "unit": "bytes"
+              "unit": "percent"
             },
             "overrides": []
           },
@@ -4267,9 +3464,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 29
+            "y": 5
           },
-          "id": 23,
+          "id": 15,
           "options": {
             "legend": {
               "calcs": [],
@@ -4288,10 +3485,12 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(jvm_memory_max_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
               "interval": "",
-              "legendFormat": "Maximum heap memory",
+              "legendFormat": "System CPU Load",
+              "range": true,
               "refId": "A"
             },
             {
@@ -4300,52 +3499,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "code",
-              "exemplar": true,
-              "expr": "sum(jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
               "hide": false,
-              "interval": "",
-              "legendFormat": "Used heap memory",
+              "legendFormat": "Process CPU Load",
               "range": true,
               "refId": "B"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Old Gen\"}",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Old area",
-              "refId": "C"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Eden Space\"}",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Eden area",
-              "refId": "D"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Survivor Space\"}",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Survivor area",
-              "refId": "E"
             }
           ],
-          "title": "Heap Memory",
+          "title": "CPU Load",
           "type": "timeseries"
         },
         {
@@ -4361,6 +3522,8 @@
               "custom": {
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 10,
@@ -4388,14 +3551,14 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
                   }
                 ]
               },
-              "unit": "bytes"
+              "unit": "ns"
             },
             "overrides": []
           },
@@ -4403,9 +3566,9 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 29
+            "y": 5
           },
-          "id": 24,
+          "id": 16,
           "options": {
             "legend": {
               "calcs": [],
@@ -4425,15 +3588,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "code",
-              "exemplar": true,
-              "expr": "sum(jvm_buffer_memory_used_bytes{instance=~\"$instance\"})",
-              "interval": "",
-              "legendFormat": "off-heap memory",
+              "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
+              "hide": false,
+              "legendFormat": "Process CPU Time",
               "range": true,
-              "refId": "A"
+              "refId": "B"
             }
           ],
-          "title": "Off Heap Memory",
+          "title": "CPU Time Per Minute",
           "type": "timeseries"
         },
         {
@@ -4459,9 +3621,6 @@
                   "viz": false
                 },
                 "lineInterpolation": "linear",
-                "lineStyle": {
-                  "fill": "solid"
-                },
                 "lineWidth": 1,
                 "pointSize": 5,
                 "scaleDistribution": {
@@ -4485,17 +3644,64 @@
                     "color": "green"
                   }
                 ]
+              },
+              "unit": "none"
+            },
+            "overrides": [
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "warn"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "trace"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "purple",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "error"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               }
-            },
-            "overrides": []
+            ]
           },
           "gridPos": {
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 29
+            "y": 5
           },
-          "id": 27,
+          "id": 25,
           "options": {
             "legend": {
               "calcs": [],
@@ -4516,28 +3722,15 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
-              "interval": "",
-              "legendFormat": "Unloaded Number",
-              "range": true,
-              "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+              "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
               "hide": false,
               "interval": "",
-              "legendFormat": "Loaded Number",
+              "legendFormat": "{{level}}",
               "range": true,
-              "refId": "B"
+              "refId": "C"
             }
           ],
-          "title": "The  Number Of Class",
+          "title": "Log Number Per Minute",
           "type": "timeseries"
         },
         {
@@ -4545,7 +3738,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -4556,7 +3748,7 @@
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 10,
+                "fillOpacity": 5,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -4588,7 +3780,7 @@
                   }
                 ]
               },
-              "unit": "ms"
+              "unit": "bytes"
             },
             "overrides": []
           },
@@ -4596,9 +3788,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 37
+            "y": 13
           },
-          "id": 26,
+          "id": 17,
           "options": {
             "legend": {
               "calcs": [],
@@ -4619,43 +3811,42 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
+              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}-sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"}",
               "interval": "",
-              "legendFormat": "compiler",
+              "legendFormat": "Used physical memory",
               "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Total physical memory",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "sys_committed_vm_size{instance=\"$instance\"}",
+              "hide": false,
+              "legendFormat": "Committed vm size",
+              "range": true,
+              "refId": "C"
             }
           ],
-          "title": "The Time Consumed of Compilation Per Minute",
+          "title": "System Memory",
           "type": "timeseries"
-        }
-      ],
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "refId": "A"
-        }
-      ],
-      "title": "System",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 24
-      },
-      "id": 29,
-      "panels": [
+        },
         {
           "datasource": {
             "type": "prometheus",
@@ -4669,7 +3860,6 @@
               "custom": {
                 "axisLabel": "",
                 "axisPlacement": "auto",
-                "axisSoftMax": 1,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 5,
@@ -4685,59 +3875,77 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
                   "mode": "none"
                 },
                 "thresholdsStyle": {
-                  "mode": "line+area"
+                  "mode": "off"
                 }
               },
               "mappings": [],
-              "max": 1,
               "thresholds": {
-                "mode": "percentage",
+                "mode": "absolute",
                 "steps": [
                   {
                     "color": "green"
-                  },
-                  {
-                    "color": "#EAB839",
-                    "value": 80
-                  },
-                  {
-                    "color": "dark-red",
-                    "value": 90
                   }
                 ]
               },
-              "unit": "percentunit"
+              "unit": "bytes"
             },
-            "overrides": []
+            "overrides": [
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Total Swap Size"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Used Swap Size"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              }
+            ]
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 0,
-            "y": 6
+            "w": 8,
+            "x": 8,
+            "y": 13
           },
-          "id": 31,
+          "id": 18,
           "options": {
             "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -4748,14 +3956,27 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "disk_io_busy_percentage{instance=~\"$instance\"}",
-              "instant": false,
+              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"} - sys_free_swap_space_size{instance=\"$instance\", name=\"system\"}",
+              "hide": false,
               "interval": "",
-              "legendFormat": "{{name}}",
-              "refId": "A"
+              "legendFormat": "Used Swap Size",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
+              "hide": false,
+              "legendFormat": "Total Swap Size",
+              "range": true,
+              "refId": "C"
             }
           ],
-          "title": "Disk I/O Busy Rate",
+          "title": "System Swap Size",
           "type": "timeseries"
         },
         {
@@ -4786,7 +4007,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -4802,35 +4023,34 @@
                 "steps": [
                   {
                     "color": "green"
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
                   }
                 ]
               },
-              "unit": "KBs"
+              "unit": "bytes"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 6
+            "w": 8,
+            "x": 16,
+            "y": 13
           },
-          "id": 33,
+          "id": 19,
           "options": {
             "legend": {
-              "calcs": [
-                "mean",
-                "last"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Last",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -4839,15 +4059,38 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "exemplar": true,
-              "expr": "rate(disk_io_size{instance=~\"$instance\"}[1m])",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
+              "editorMode": "code",
+              "expr": "process_max_mem{instance=~\"${instance}\"}",
+              "legendFormat": "Max Memory",
+              "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "process_total_mem{instance=~\"${instance}\"}",
+              "hide": false,
+              "legendFormat": "Total Memory",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "process_total_mem{instance=~\"${instance}\"} - process_free_mem{instance=~\"${instance}\"}",
+              "hide": false,
+              "legendFormat": "Used Memory",
+              "range": true,
+              "refId": "C"
             }
           ],
-          "title": "Disk I/O Throughput",
+          "title": "Process Memory",
           "type": "timeseries"
         },
         {
@@ -4878,7 +4121,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -4896,33 +4139,27 @@
                     "color": "green"
                   }
                 ]
-              },
-              "unit": "iops"
+              }
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
+            "w": 8,
             "x": 0,
-            "y": 14
+            "y": 21
           },
-          "id": 35,
+          "id": 20,
           "options": {
             "legend": {
-              "calcs": [
-                "mean",
-                "max"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Max",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -4931,14 +4168,28 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "rate(disk_io_ops{instance=~\"$instance\"}[1m])",
+              "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
+              "legendFormat": "Young GC number",
+              "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Full GC number",
+              "refId": "B"
             }
           ],
-          "title": "Disk I/O Ops",
+          "title": "The Number Of GC Per Minute",
           "type": "timeseries"
         },
         {
@@ -4956,7 +4207,7 @@
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -4969,7 +4220,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -4988,27 +4239,27 @@
                   }
                 ]
               },
-              "unit": "ms"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 14
+            "w": 8,
+            "x": 8,
+            "y": 21
           },
-          "id": 37,
+          "id": 21,
           "options": {
             "legend": {
               "calcs": [],
-              "displayMode": "table",
-              "placement": "right",
+              "displayMode": "list",
+              "placement": "bottom",
               "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -5018,13 +4269,25 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(disk_io_time{instance=~\"$instance\"}[1m])",
+              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
+              "legendFormat": "The time of young GC",
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "The time of full GC",
+              "refId": "B"
             }
           ],
-          "title": "Disk I/O Time",
+          "title": "The Time Consumed Of GC Per Minute",
           "type": "timeseries"
         },
         {
@@ -5032,7 +4295,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -5056,7 +4318,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -5074,33 +4336,27 @@
                     "color": "green"
                   }
                 ]
-              },
-              "unit": "ms"
+              }
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 8,
-            "x": 0,
-            "y": 22
+            "x": 16,
+            "y": 21
           },
-          "id": 39,
+          "id": 22,
           "options": {
             "legend": {
-              "calcs": [
-                "mean",
-                "max"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Max",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -5111,14 +4367,26 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "disk_io_avg_time{instance=~\"$instance\"}",
+              "expr": "jvm_threads_live_threads{instance=~\"$instance\"}",
               "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
+              "legendFormat": "Total Number",
               "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "jvm_threads_states_threads{instance=~\"$instance\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "{{state}}",
+              "refId": "B"
             }
           ],
-          "title": "Disk Per I/O Avg Time",
+          "title": "The Number Of Java Thread",
           "type": "timeseries"
         },
         {
@@ -5136,7 +4404,7 @@
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -5149,7 +4417,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -5175,24 +4443,20 @@
           "gridPos": {
             "h": 8,
             "w": 8,
-            "x": 8,
-            "y": 22
+            "x": 0,
+            "y": 29
           },
-          "id": 41,
+          "id": 23,
           "options": {
             "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -5202,13 +4466,63 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "disk_io_avg_size{instance=~\"$instance\"}",
+              "expr": "sum(jvm_memory_max_bytes{instance=~\"$instance\",area=\"heap\"})",
               "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
+              "legendFormat": "Maximum heap memory",
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sum(jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Used heap memory",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Old Gen\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Old area",
+              "refId": "C"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Eden Space\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Eden area",
+              "refId": "D"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Survivor Space\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Survivor area",
+              "refId": "E"
             }
           ],
-          "title": "Disk I/O Avg Size",
+          "title": "Heap Memory",
           "type": "timeseries"
         },
         {
@@ -5226,7 +4540,7 @@
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -5239,7 +4553,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -5257,31 +4571,28 @@
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "bytes"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 8,
-            "x": 16,
-            "y": 22
+            "x": 8,
+            "y": 29
           },
-          "id": 43,
+          "id": 24,
           "options": {
             "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -5292,14 +4603,14 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "disk_io_avg_queue_size{instance=~\"$instance\"}",
+              "expr": "sum(jvm_buffer_memory_used_bytes{instance=~\"$instance\"})",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "off-heap memory",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "Disk I/O Avg Queue Size",
+          "title": "Off Heap Memory",
           "type": "timeseries"
         },
         {
@@ -5317,7 +4628,7 @@
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -5325,12 +4636,15 @@
                   "viz": false
                 },
                 "lineInterpolation": "linear",
+                "lineStyle": {
+                  "fill": "solid"
+                },
                 "lineWidth": 1,
                 "pointSize": 5,
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -5348,30 +4662,27 @@
                     "color": "green"
                   }
                 ]
-              },
-              "unit": "KBs"
+              }
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 0,
-            "y": 30
+            "w": 8,
+            "x": 16,
+            "y": 29
           },
-          "id": 47,
+          "id": 27,
           "options": {
             "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
               "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -5380,14 +4691,30 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "rate(process_io_size{instance=~\"$instance\"}[1m])",
+              "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
               "interval": "",
-              "legendFormat": "{{from}}-{{name}}",
+              "legendFormat": "Unloaded Number",
+              "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Loaded Number",
+              "range": true,
+              "refId": "B"
             }
           ],
-          "title": "Process I/O Throughput",
+          "title": "The  Number Of Class",
           "type": "timeseries"
         },
         {
@@ -5395,6 +4722,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -5405,7 +4733,7 @@
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 5,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -5418,7 +4746,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -5437,31 +4765,27 @@
                   }
                 ]
               },
-              "unit": "ops"
+              "unit": "ms"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 30
+            "w": 8,
+            "x": 0,
+            "y": 37
           },
-          "id": 45,
+          "id": 26,
           "options": {
             "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -5470,14 +4794,16 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "rate(process_io_ops{instance=~\"$instance\"}[1m])",
+              "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "{{from}}-{{name}}",
+              "legendFormat": "compiler",
+              "range": true,
               "refId": "A"
             }
           ],
-          "title": "I/O System Call Rate",
+          "title": "The Time Consumed of Compilation Per Minute",
           "type": "timeseries"
         }
       ],
@@ -5485,17 +4811,17 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
       ],
-      "title": "Disk I/O",
+      "title": "System",
       "type": "row"
     }
   ],
-  "refresh": false,
-  "schemaVersion": 35,
+  "refresh": "15s",
+  "schemaVersion": 37,
   "style": "dark",
   "tags": [
     "Apache-IoTDB",
@@ -5567,6 +4893,6 @@
   "timezone": "browser",
   "title": "Apache IoTDB ConfigNode Dashboard",
   "uid": "4WkTYkx4z",
-  "version": 15,
+  "version": 2,
   "weekStart": ""
 }
\ No newline at end of file
diff --git a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
index bdbe5ec5a3..dc79eb7d61 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
@@ -9,7 +9,7 @@
       "pluginName": "Prometheus"
     }
   ],
-  "__elements": [],
+  "__elements": {},
   "__requires": [
     {
       "type": "panel",
@@ -21,7 +21,7 @@
       "type": "grafana",
       "id": "grafana",
       "name": "Grafana",
-      "version": "8.4.2"
+      "version": "9.3.6"
     },
     {
       "type": "panel",
@@ -75,7 +75,6 @@
   "fiscalYearStartMonth": 0,
   "graphTooltip": 1,
   "id": null,
-  "iteration": 1679118029653,
   "links": [],
   "liveNow": false,
   "panels": [
@@ -83,7 +82,7 @@
       "collapsed": false,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -97,7 +96,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -116,6 +115,8 @@
             "mode": "palette-classic"
           },
           "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
             "axisLabel": "",
             "axisPlacement": "auto",
             "barAlignment": 0,
@@ -204,6 +205,8 @@
             "mode": "palette-classic"
           },
           "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
             "axisLabel": "",
             "axisPlacement": "auto",
             "barAlignment": 0,
@@ -292,6 +295,8 @@
             "mode": "palette-classic"
           },
           "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
             "axisLabel": "",
             "axisPlacement": "auto",
             "barAlignment": 0,
@@ -374,7 +379,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -1116,7 +1121,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -1128,7 +1133,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -1842,7 +1847,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -1854,7 +1859,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -5311,7 +5316,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -5323,7 +5328,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -9199,7 +9204,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -9211,7 +9216,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -10986,7 +10991,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -10998,7 +11003,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -12358,7 +12363,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -12368,6 +12373,10 @@
     },
     {
       "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -12377,6 +12386,10 @@
       "id": 238,
       "panels": [
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -12461,7 +12474,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -12522,6 +12536,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -12606,7 +12624,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -12666,6 +12685,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -12750,7 +12773,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -12787,6 +12811,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -12871,7 +12899,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -12908,6 +12937,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -12971,7 +13004,8 @@
                 "mean"
               ],
               "displayMode": "list",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -13007,11 +13041,24 @@
           "type": "timeseries"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "DataRegion Ratis Consensus",
       "type": "row"
     },
     {
       "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "Ar7cRMx4z"
+      },
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -13021,6 +13068,10 @@
       "id": 240,
       "panels": [
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -13105,7 +13156,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -13166,6 +13218,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -13250,7 +13306,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -13310,6 +13367,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -13394,7 +13455,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -13431,6 +13493,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -13515,7 +13581,8 @@
                 "mean"
               ],
               "displayMode": "table",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -13552,6 +13619,10 @@
           "type": "timeseries"
         },
         {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -13613,7 +13684,8 @@
             "legend": {
               "calcs": [],
               "displayMode": "list",
-              "placement": "right"
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "multi",
@@ -13649,6 +13721,15 @@
           "type": "timeseries"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "SchemaRegion Ratis Consensus",
       "type": "row"
     },
@@ -13656,7 +13737,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -15449,7 +15530,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "Ar7cRMx4z"
           },
           "refId": "A"
         }
@@ -15461,7 +15542,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
+        "uid": "Ar7cRMx4z"
       },
       "gridPos": {
         "h": 1,
@@ -15469,7 +15550,7 @@
         "x": 0,
         "y": 18
       },
-      "id": 177,
+      "id": 197,
       "panels": [
         {
           "datasource": {
@@ -15479,98 +15560,74 @@
           "fieldConfig": {
             "defaults": {
               "color": {
-                "mode": "palette-classic"
+                "mode": "thresholds"
               },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 1,
-                "axisSoftMin": 0,
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 10,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "line+area"
+              "mappings": [
+                {
+                  "options": {
+                    "0": {
+                      "index": 0,
+                      "text": "Memory"
+                    },
+                    "1": {
+                      "index": 1,
+                      "text": "Schema_File"
+                    }
+                  },
+                  "type": "value"
                 }
-              },
-              "mappings": [],
+              ],
               "thresholds": {
-                "mode": "percentage",
+                "mode": "absolute",
                 "steps": [
                   {
                     "color": "green"
-                  },
-                  {
-                    "color": "#EAB839",
-                    "value": 80
-                  },
-                  {
-                    "color": "red",
-                    "value": 90
                   }
                 ]
               },
-              "unit": "percentunit"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
-            "w": 12,
+            "h": 4,
+            "w": 8,
             "x": 0,
-            "y": 95
+            "y": 12
           },
-          "id": 179,
+          "id": 224,
           "options": {
-            "legend": {
+            "colorMode": "none",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
               "calcs": [
-                "mean",
-                "last"
+                "lastNotNull"
               ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
+              "fields": "",
+              "values": false
             },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
+            "textMode": "value"
           },
+          "pluginVersion": "8.4.2",
           "targets": [
             {
               "datasource": {
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "exemplar": true,
-              "expr": "disk_io_busy_percentage{instance=~\"$instance\"}/10000",
-              "interval": "",
+              "editorMode": "code",
+              "exemplar": false,
+              "expr": "schema_engine{name=\"schema_engine_mode\",instance=~\"$instance\"}",
+              "instant": false,
               "legendFormat": "{{name}}",
+              "range": true,
               "refId": "A"
             }
           ],
-          "title": "Disk I/O Busy Rate",
-          "type": "timeseries"
+          "title": "Schema Engine Mode",
+          "type": "stat"
         },
         {
           "datasource": {
@@ -15580,39 +15637,23 @@
           "fieldConfig": {
             "defaults": {
               "color": {
-                "mode": "palette-classic"
+                "mode": "thresholds"
               },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
+              "mappings": [
+                {
+                  "options": {
+                    "0": {
+                      "index": 0,
+                      "text": "RatisConsensus"
+                    },
+                    "1": {
+                      "index": 1,
+                      "text": "SimpleConsensus"
+                    }
+                  },
+                  "type": "value"
                 }
-              },
-              "mappings": [],
+              ],
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
@@ -15621,843 +15662,28 @@
                   }
                 ]
               },
-              "unit": "KBs"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 95
+            "h": 4,
+            "w": 8,
+            "x": 8,
+            "y": 12
           },
-          "id": 181,
+          "id": 225,
           "options": {
-            "legend": {
+            "colorMode": "none",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
               "calcs": [
-                "mean",
-                "last"
+                "lastNotNull"
               ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "none",
-              "sort": "none"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "rate(disk_io_size{instance=~\"$instance\"}[1m])",
-              "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
-              "refId": "A"
-            }
-          ],
-          "title": "Disk I/O Throughput",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "iops"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 12,
-            "x": 0,
-            "y": 103
-          },
-          "id": 183,
-          "options": {
-            "legend": {
-              "calcs": [
-                "mean",
-                "max"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "rate(disk_io_ops{instance=~\"$instance\"}[1m])",
-              "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
-              "refId": "A"
-            }
-          ],
-          "title": "Disk I/O Ops",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "ms"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 103
-          },
-          "id": 185,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "rate(disk_io_time{instance=~\"$instance\"}[1m])",
-              "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
-              "refId": "A"
-            }
-          ],
-          "title": "Disk I/O Time",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "decimals": 2,
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "ms"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 111
-          },
-          "id": 187,
-          "options": {
-            "legend": {
-              "calcs": [
-                "mean",
-                "max"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "disk_io_avg_time{instance=~\"$instance\"}/1000000",
-              "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
-              "refId": "A"
-            }
-          ],
-          "title": "Disk Per I/O Avg Time",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "bytes"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 111
-          },
-          "id": 189,
-          "options": {
-            "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "disk_io_avg_size{instance=~\"$instance\"}",
-              "interval": "",
-              "legendFormat": "{{name}}-{{type}}",
-              "refId": "A"
-            }
-          ],
-          "title": "Disk I/O Avg Size",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              }
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 111
-          },
-          "id": 193,
-          "options": {
-            "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "sortBy": "Mean",
-              "sortDesc": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "disk_io_queue_avg_size{instance=~\"$instance\"}/10000",
-              "interval": "",
-              "legendFormat": "{{name}}",
-              "range": true,
-              "refId": "A"
-            }
-          ],
-          "title": "Disk I/O Avg Queue Size",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "ops"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 12,
-            "x": 0,
-            "y": 119
-          },
-          "id": 191,
-          "options": {
-            "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "rate(process_io_ops{instance=~\"$instance\"}[1m])",
-              "interval": "",
-              "legendFormat": "{{from}}-{{name}}",
-              "refId": "A"
-            }
-          ],
-          "title": "I/O System Call Rate",
-          "type": "timeseries"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 5,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "auto",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "KBs"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 119
-          },
-          "id": 195,
-          "options": {
-            "legend": {
-              "calcs": [
-                "mean"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "asc"
-            }
-          },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "rate(process_io_size{instance=~\"$instance\"}[1m])",
-              "interval": "",
-              "legendFormat": "{{from}}-{{name}}",
-              "refId": "A"
-            }
-          ],
-          "title": "Process I/O Throughput",
-          "type": "timeseries"
-        }
-      ],
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "refId": "A"
-        }
-      ],
-      "title": "Disk Status",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 19
-      },
-      "id": 197,
-      "panels": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "thresholds"
-              },
-              "mappings": [
-                {
-                  "options": {
-                    "0": {
-                      "index": 0,
-                      "text": "Memory"
-                    },
-                    "1": {
-                      "index": 1,
-                      "text": "Schema_File"
-                    }
-                  },
-                  "type": "value"
-                }
-              ],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "none"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 4,
-            "w": 8,
-            "x": 0,
-            "y": 12
-          },
-          "id": 224,
-          "options": {
-            "colorMode": "none",
-            "graphMode": "none",
-            "justifyMode": "auto",
-            "orientation": "auto",
-            "reduceOptions": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "fields": "",
-              "values": false
-            },
-            "textMode": "value"
-          },
-          "pluginVersion": "8.4.2",
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": false,
-              "expr": "schema_engine{name=\"schema_engine_mode\",instance=~\"$instance\"}",
-              "instant": false,
-              "legendFormat": "{{name}}",
-              "range": true,
-              "refId": "A"
-            }
-          ],
-          "title": "Schema Engine Mode",
-          "type": "stat"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "thresholds"
-              },
-              "mappings": [
-                {
-                  "options": {
-                    "0": {
-                      "index": 0,
-                      "text": "RatisConsensus"
-                    },
-                    "1": {
-                      "index": 1,
-                      "text": "SimpleConsensus"
-                    }
-                  },
-                  "type": "value"
-                }
-              ],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "none"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 4,
-            "w": 8,
-            "x": 8,
-            "y": 12
-          },
-          "id": 225,
-          "options": {
-            "colorMode": "none",
-            "graphMode": "none",
-            "justifyMode": "auto",
-            "orientation": "auto",
-            "reduceOptions": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "fields": "",
-              "values": false
+              "fields": "",
+              "values": false
             },
             "textMode": "value"
           },
@@ -18328,12 +17554,21 @@
           "type": "timeseries"
         }
       ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "Ar7cRMx4z"
+          },
+          "refId": "A"
+        }
+      ],
       "title": "Schema Engine",
       "type": "row"
     }
   ],
-  "refresh": false,
-  "schemaVersion": 35,
+  "refresh": "15s",
+  "schemaVersion": 37,
   "style": "dark",
   "tags": [
     "Apache-IoTDB",
@@ -18405,6 +17640,6 @@
   "timezone": "browser",
   "title": "Apache IoTDB DataNode Dashboard",
   "uid": "TbEVYRw7A",
-  "version": 5,
+  "version": 2,
   "weekStart": ""
 }
\ No newline at end of file
diff --git a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-Disk-IO-Dashboard.json b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-Disk-IO-Dashboard.json
new file mode 100644
index 0000000000..dc97a26edf
--- /dev/null
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-Disk-IO-Dashboard.json
@@ -0,0 +1,894 @@
+{
+  "__inputs": [
+    {
+      "name": "DS_PROMETHEUS",
+      "label": "Prometheus",
+      "description": "",
+      "type": "datasource",
+      "pluginId": "prometheus",
+      "pluginName": "Prometheus"
+    }
+  ],
+  "__elements": {},
+  "__requires": [
+    {
+      "type": "grafana",
+      "id": "grafana",
+      "name": "Grafana",
+      "version": "9.3.6"
+    },
+    {
+      "type": "datasource",
+      "id": "prometheus",
+      "name": "Prometheus",
+      "version": "1.0.0"
+    },
+    {
+      "type": "panel",
+      "id": "timeseries",
+      "name": "Time series",
+      "version": ""
+    }
+  ],
+  "annotations": {
+    "list": [
+      {
+        "builtIn": 1,
+        "datasource": {
+          "type": "grafana",
+          "uid": "-- Grafana --"
+        },
+        "enable": true,
+        "hide": true,
+        "iconColor": "rgba(0, 211, 255, 1)",
+        "name": "Annotations & Alerts",
+        "target": {
+          "limit": 100,
+          "matchAny": false,
+          "tags": [],
+          "type": "dashboard"
+        },
+        "type": "dashboard"
+      }
+    ]
+  },
+  "editable": true,
+  "fiscalYearStartMonth": 0,
+  "graphTooltip": 0,
+  "id": null,
+  "links": [],
+  "liveNow": false,
+  "panels": [
+    {
+      "collapsed": false,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 0
+      },
+      "id": 18,
+      "panels": [],
+      "title": "Disk Status",
+      "type": "row"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "axisSoftMax": 1,
+            "axisSoftMin": 0,
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "line+area"
+            }
+          },
+          "mappings": [],
+          "max": 1,
+          "min": 0,
+          "thresholds": {
+            "mode": "percentage",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "percentunit"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 1
+      },
+      "id": 2,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "disk_io_busy_percentage{instance=~\"$instance\", disk_id=~\"$disk\"}",
+          "legendFormat": "{{disk_id}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Disk I/O Busy Rate",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          },
+          "unit": "KBs"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 1
+      },
+      "id": 4,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "rate(disk_io_size{instance=~\"$instance\", disk_id=~\"$disk\"}[1m])",
+          "legendFormat": "{{disk_id}}-{{type}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Disk I/O Throughput",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          },
+          "unit": "iops"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 9
+      },
+      "id": 6,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "rate(disk_io_ops{instance=~\"$instance\", disk_id=~\"$disk\"}[1m])",
+          "legendFormat": "{{disk_id}}-{{type}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Disk I/O Ops",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          },
+          "unit": "ms"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 9
+      },
+      "id": 10,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "disk_io_avg_time{instance=~\"$instance\", disk_id=~\"$disk\"}",
+          "legendFormat": "{{disk_id}}-{{type}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Disk Per I/O Avg Time",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          },
+          "unit": "bytes"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 17
+      },
+      "id": 12,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "disk_io_avg_size{instance=~\"$instance\", disk_id=~\"$disk\"}",
+          "legendFormat": "{{disk_id}}-{{type}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Disk I/O Avg Size",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 17
+      },
+      "id": 8,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "disk_io_avg_queue_size{instance=~\"$instance\", disk_id=~\"$disk\"}",
+          "legendFormat": "{{disk_id}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Disk I/O Avg Queue Size",
+      "type": "timeseries"
+    },
+    {
+      "collapsed": false,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 25
+      },
+      "id": 20,
+      "panels": [],
+      "title": "Process",
+      "type": "row"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green"
+              }
+            ]
+          },
+          "unit": "KBs"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 26
+      },
+      "id": 16,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "rate(process_io_size{instance=~\"$instance\"}[1m])",
+          "legendFormat": "{{from}}-{{name}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Process I/O Throughput",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisCenteredZero": false,
+            "axisColorMode": "text",
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green"
+              }
+            ]
+          },
+          "unit": "ops"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 26
+      },
+      "id": 14,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right",
+          "showLegend": true
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "asc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "expr": "rate(process_io_ops{instance=~\"$instance\"}[1m])",
+          "legendFormat": "{{from}}-{{name}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "I/O System Call Rate",
+      "type": "timeseries"
+    }
+  ],
+  "refresh": "15s",
+  "schemaVersion": 37,
+  "style": "dark",
+  "tags": [],
+  "templating": {
+    "list": [
+      {
+        "current": {},
+        "datasource": {
+          "type": "prometheus",
+          "uid": "${DS_PROMETHEUS}"
+        },
+        "definition": "label_values(instance)",
+        "description": "",
+        "hide": 0,
+        "includeAll": false,
+        "multi": false,
+        "name": "instance",
+        "options": [],
+        "query": {
+          "query": "label_values(instance)",
+          "refId": "StandardVariableQuery"
+        },
+        "refresh": 1,
+        "regex": "",
+        "skipUrlSync": false,
+        "sort": 0,
+        "type": "query"
+      },
+      {
+        "current": {},
+        "datasource": {
+          "type": "prometheus",
+          "uid": "${DS_PROMETHEUS}"
+        },
+        "definition": "label_values(disk_id)",
+        "hide": 0,
+        "includeAll": false,
+        "multi": true,
+        "name": "disk",
+        "options": [],
+        "query": {
+          "query": "label_values(disk_id)",
+          "refId": "StandardVariableQuery"
+        },
+        "refresh": 1,
+        "regex": "",
+        "skipUrlSync": false,
+        "sort": 0,
+        "type": "query"
+      }
+    ]
+  },
+  "time": {
+    "from": "2023-03-22T12:12:11.190Z",
+    "to": "2023-03-22T12:22:11.190Z"
+  },
+  "timepicker": {
+    "refresh_intervals": [
+      "15s",
+      "30s",
+      "1m",
+      "5m",
+      "15m",
+      "30m"
+    ]
+  },
+  "timezone": "",
+  "title": "Apache IoTDB Disk I/O Dashboard",
+  "uid": "TrJ0dsaVk",
+  "version": 2,
+  "weekStart": ""
+}
\ No newline at end of file
diff --git a/metrics/interface/src/main/java/org/apache/iotdb/metrics/metricsets/disk/DiskMetrics.java b/metrics/interface/src/main/java/org/apache/iotdb/metrics/metricsets/disk/DiskMetrics.java
index 00d550be49..d9ff06f6dd 100644
--- a/metrics/interface/src/main/java/org/apache/iotdb/metrics/metricsets/disk/DiskMetrics.java
+++ b/metrics/interface/src/main/java/org/apache/iotdb/metrics/metricsets/disk/DiskMetrics.java
@@ -42,8 +42,10 @@ public class DiskMetrics implements IMetricSet {
   private static final String ACTUAL_WRITE = "actual_write";
   private static final String ACTUAL_READ = "actual_read";
   private static final String TYPE = "type";
+  private static final String DISK_ID = "disk_id";
   private static final String NAME = "name";
   private static final String FROM = "from";
+
   private static final String DISK_IO_SIZE = "disk_io_size";
   private static final String DISK_IO_OPS = "disk_io_ops";
   private static final String DISK_IO_TIME = "disk_io_time";
@@ -71,7 +73,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getReadDataSizeForDisk().getOrDefault(diskID, 0.0),
           TYPE,
           READ,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_SIZE,
@@ -80,7 +82,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getWriteDataSizeForDisk().getOrDefault(diskID, 0.0),
           TYPE,
           WRITE,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_OPS,
@@ -89,7 +91,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getReadOperationCountForDisk().getOrDefault(diskID, 0L),
           TYPE,
           READ,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_OPS,
@@ -98,7 +100,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getWriteOperationCountForDisk().getOrDefault(diskID, 0L),
           TYPE,
           WRITE,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_OPS,
@@ -106,8 +108,8 @@ public class DiskMetrics implements IMetricSet {
           diskMetricsManager,
           x -> x.getMergedReadOperationForDisk().getOrDefault(diskID, 0L),
           TYPE,
-          MERGED_WRITE,
-          NAME,
+          MERGED_READ,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_OPS,
@@ -115,8 +117,8 @@ public class DiskMetrics implements IMetricSet {
           diskMetricsManager,
           x -> x.getMergedWriteOperationForDisk().getOrDefault(diskID, 0L),
           TYPE,
-          MERGED_READ,
-          NAME,
+          MERGED_WRITE,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_TIME,
@@ -125,7 +127,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getReadCostTimeForDisk().getOrDefault(diskID, 0L),
           TYPE,
           READ,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_TIME,
@@ -134,7 +136,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getWriteCostTimeForDisk().getOrDefault(diskID, 0L),
           TYPE,
           WRITE,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_AVG_TIME,
@@ -143,7 +145,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getAvgReadCostTimeOfEachOpsForDisk().getOrDefault(diskID, 0.0),
           TYPE,
           READ,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_AVG_TIME,
@@ -152,7 +154,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getAvgWriteCostTimeOfEachOpsForDisk().getOrDefault(diskID, 0.0),
           TYPE,
           WRITE,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_AVG_SIZE,
@@ -161,7 +163,7 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getAvgSizeOfEachReadForDisk().getOrDefault(diskID, 0.0),
           TYPE,
           READ,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_AVG_SIZE,
@@ -170,21 +172,21 @@ public class DiskMetrics implements IMetricSet {
           x -> x.getAvgSizeOfEachWriteForDisk().getOrDefault(diskID, 0.0),
           TYPE,
           WRITE,
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_BUSY_PERCENTAGE,
           MetricLevel.IMPORTANT,
           diskMetricsManager,
           x -> x.getIoUtilsPercentage().getOrDefault(diskID, 0.0),
-          NAME,
+          DISK_ID,
           diskID);
       metricService.createAutoGauge(
           DISK_IO_QUEUE_SIZE,
           MetricLevel.IMPORTANT,
           diskMetricsManager,
           x -> x.getQueueSizeForDisk().getOrDefault(diskID, 0.0),
-          NAME,
+          DISK_ID,
           diskID);
     }
 
@@ -250,16 +252,18 @@ public class DiskMetrics implements IMetricSet {
     // metrics for disks
     Set<String> diskIDs = diskMetricsManager.getDiskIds();
     for (String diskID : diskIDs) {
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_SIZE, NAME, READ, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_SIZE, NAME, WRITE, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_OPS, NAME, READ, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_OPS, NAME, WRITE, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, NAME, READ, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, NAME, WRITE, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, NAME, AVG_READ, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, NAME, AVG_WRITE, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_AVG_SIZE, NAME, READ, NAME, diskID);
-      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_AVG_SIZE, NAME, WRITE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_SIZE, TYPE, READ, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_SIZE, TYPE, WRITE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_OPS, TYPE, READ, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_OPS, TYPE, WRITE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, TYPE, READ, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, TYPE, WRITE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, TYPE, AVG_READ, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_TIME, TYPE, AVG_WRITE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_AVG_SIZE, TYPE, READ, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_AVG_SIZE, TYPE, WRITE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_BUSY_PERCENTAGE, NAME, diskID);
+      metricService.remove(MetricType.AUTO_GAUGE, DISK_IO_QUEUE_SIZE, NAME, diskID);
     }
 
     // metrics for datanode and config node


[iotdb] 02/04: [To rel/1.1] Correct CQ example in docs (#9407)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 9f6e0e72bf621a4785aa5bc789cec0fd1eedbba9
Author: Jackie Tien <ja...@gmail.com>
AuthorDate: Wed Mar 22 17:19:01 2023 +0800

    [To rel/1.1] Correct CQ example in docs (#9407)
---
 docs/UserGuide/Administration-Management/Administration.md    | 2 +-
 docs/zh/UserGuide/Administration-Management/Administration.md | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/UserGuide/Administration-Management/Administration.md b/docs/UserGuide/Administration-Management/Administration.md
index 4b3b7c3c71..2bf7d88242 100644
--- a/docs/UserGuide/Administration-Management/Administration.md
+++ b/docs/UserGuide/Administration-Management/Administration.md
@@ -391,7 +391,7 @@ At the same time, changes to roles are immediately reflected on all users who ow
 | DROP_FUNCTION             | deregister UDFs; path independent                                                                                              | Eg: `drop function example`                                                                                                                                                                                                                                                                                                                   [...]
 | CREATE_TRIGGER            | create triggers; path dependent                                                                                                | Eg1: `CREATE TRIGGER <TRIGGER-NAME> BEFORE INSERT ON <FULL-PATH> AS <CLASSNAME>`<br />Eg2: `CREATE TRIGGER <TRIGGER-NAME> AFTER INSERT ON <FULL-PATH> AS <CLASSNAME>`                                                                                                                                                                         [...]
 | DROP_TRIGGER              | drop triggers; path dependent                                                                                                  | Eg: `drop trigger 'alert-listener-sg1d1s1'`                                                                                                                                                                                                                                                                                                   [...]
-| CREATE_CONTINUOUS_QUERY   | create continuous queries; path independent                                                                                    | Eg: `select s1, s1 into t1, t2 from root.sg.d1`                                                                                                                                                                                                                                                                                               [...]
+| CREATE_CONTINUOUS_QUERY   | create continuous queries; path independent                                                                                    | Eg: `CREATE CONTINUOUS QUERY cq1 RESAMPLE RANGE 40s BEGIN <QUERY-BODY> END`                                                                                                                                                                                                                                                                   [...]
 | DROP_CONTINUOUS_QUERY     | drop continuous queries; path independent                                                                                      | Eg1: `DROP CONTINUOUS QUERY cq3`<br />Eg2: `DROP CQ cq3`                                                                                                                                                                                                                                                                                      [...]
 | SHOW_CONTINUOUS_QUERIES   | show continuous queries; path independent                                                                                      | Eg1: `SHOW CONTINUOUS QUERIES`<br />Eg2: `SHOW cqs`                                                                                                                                                                                                                                                                                           [...]
 | UPDATE_TEMPLATE           | create and drop schema template; path independent                                                                              | Eg1: `create schema template t1(s1 int32)`<br />Eg2: `drop schema template t1`                                                                                                                                                                                                                                                                [...]
diff --git a/docs/zh/UserGuide/Administration-Management/Administration.md b/docs/zh/UserGuide/Administration-Management/Administration.md
index 60a1d3d60e..3f1efa9663 100644
--- a/docs/zh/UserGuide/Administration-Management/Administration.md
+++ b/docs/zh/UserGuide/Administration-Management/Administration.md
@@ -390,7 +390,7 @@ Eg: IoTDB > ALTER USER `tempuser` SET PASSWORD 'newpwd';
 | DROP_FUNCTION             | 卸载 UDF。路径无关                             | Eg: `drop function example`                                                                                                                                                                                                                                                                                                                                                                                                          [...]
 | CREATE_TRIGGER            | 创建触发器。路径相关                              | Eg1: `CREATE TRIGGER <TRIGGER-NAME> BEFORE INSERT ON <FULL-PATH> AS <CLASSNAME>`<br />Eg2: `CREATE TRIGGER <TRIGGER-NAME> AFTER INSERT ON <FULL-PATH> AS <CLASSNAME>`                                                                                                                                                                                                                                                                [...]
 | DROP_TRIGGER              | 卸载触发器。路径相关                              | Eg: `drop trigger 'alert-listener-sg1d1s1'`                                                                                                                                                                                                                                                                                                                                                                                          [...]
-| CREATE_CONTINUOUS_QUERY   | 创建连续查询。路径无关                             | Eg: `select s1, s1 into t1, t2 from root.sg.d1`                                                                                                                                                                                                                                                                                                                                                                                      [...]
+| CREATE_CONTINUOUS_QUERY   | 创建连续查询。路径无关                             | Eg: `CREATE CONTINUOUS QUERY cq1 RESAMPLE RANGE 40s BEGIN <QUERY-BODY> END`                                                                                                                                                                                                                                                                                                                                                          [...]
 | DROP_CONTINUOUS_QUERY     | 卸载连续查询。路径无关                             | Eg1: `DROP CONTINUOUS QUERY cq3`<br />Eg2: `DROP CQ cq3`                                                                                                                                                                                                                                                                                                                                                                             [...]
 | SHOW_CONTINUOUS_QUERIES   | 展示所有连续查询。路径无关                           | Eg1: `SHOW CONTINUOUS QUERIES`<br />Eg2: `SHOW cqs`                                                                                                                                                                                                                                                                                                                                                                                  [...]
 | UPDATE_TEMPLATE           | 创建、删除模板。路径无关。                           | Eg1: `create schema template t1(s1 int32)`<br />Eg2: `drop schema template t1`                                                                                                                                                                                                                                                                                                                                                       [...]


[iotdb] 04/04: Compatible with the client parameters on DataNode (#9421)

Posted by zy...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zyk pushed a commit to branch rc/1.1.0
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 9fda3fb9f6498c12f9794b2aafba1a7aba26a05f
Author: YongzaoDan <33...@users.noreply.github.com>
AuthorDate: Wed Mar 22 23:17:53 2023 +0800

    Compatible with the client parameters on DataNode  (#9421)
---
 .../main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java  | 12 ++++--------
 1 file changed, 4 insertions(+), 8 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 29a9caf080..77378e7f72 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -229,8 +229,7 @@ public class IoTDBDescriptor {
                     "dn_connection_timeout_ms", String.valueOf(conf.getConnectionTimeoutInMS()))
                 .trim()));
 
-    if (properties.getProperty("dn_core_client_count_for_each_node_in_client_manager", null)
-        != null) {
+    if (properties.getProperty("dn_core_connection_for_internal_service", null) != null) {
       conf.setCoreClientNumForEachNode(
           Integer.parseInt(
               properties.getProperty("dn_core_connection_for_internal_service").trim()));
@@ -245,15 +244,12 @@ public class IoTDBDescriptor {
                     String.valueOf(conf.getCoreClientNumForEachNode()))
                 .trim()));
 
-    if (properties.getProperty("dn_max_client_count_for_each_node_in_client_manager", null)
-        != null) {
+    if (properties.getProperty("dn_max_connection_for_internal_service", null) != null) {
       conf.setMaxClientNumForEachNode(
           Integer.parseInt(
-              properties
-                  .getProperty("dn_max_client_count_for_each_node_in_client_manager")
-                  .trim()));
+              properties.getProperty("dn_max_connection_for_internal_service").trim()));
       logger.warn(
-          "The parameter dn_max_client_count_for_each_node_in_client_manager is out of date. Please rename it to dn_max_client_count_for_each_node_in_client_manager.");
+          "The parameter dn_max_connection_for_internal_service is out of date. Please rename it to dn_max_client_count_for_each_node_in_client_manager.");
     }
     conf.setMaxClientNumForEachNode(
         Integer.parseInt(