You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ta...@apache.org on 2023/03/17 05:33:45 UTC

[iotdb] branch master updated: [IOTDB-5618] Add RatisConsensus metrics in dashboard (#9212)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 387404d1e3 [IOTDB-5618] Add RatisConsensus metrics in dashboard (#9212)
387404d1e3 is described below

commit 387404d1e3d01a9f1ab6bcae83b391e6dcb474c1
Author: Xiangpeng Hu <65...@users.noreply.github.com>
AuthorDate: Fri Mar 17 13:33:40 2023 +0800

    [IOTDB-5618] Add RatisConsensus metrics in dashboard (#9212)
---
 .../manager/consensus/ConsensusManager.java        |    3 -
 .../iotdb/consensus/config/ConsensusConfig.java    |   15 -
 .../ratis/ApplicationStateMachineProxy.java        |   30 +-
 .../iotdb/consensus/ratis/RatisConsensus.java      |   31 +-
 .../org/apache/iotdb/consensus/ratis/Utils.java    |   15 +
 .../ratis/metrics/IoTDBMetricRegistry.java         |   65 +-
 .../ratis/metrics/MetricRegistryManager.java       |   13 +-
 .../consensus/ratis/metrics/RatisMetricSet.java    |   79 +-
 .../ratis/metrics/RatisMetricsManager.java         |   84 +
 .../apache/iotdb/consensus/iot/ReplicateTest.java  |    2 -
 .../apache/iotdb/consensus/iot/StabilityTest.java  |    2 -
 .../iotdb/consensus/ratis/RatisConsensusTest.java  |    2 -
 .../apache/iotdb/consensus/ratis/SnapshotTest.java |   11 +-
 .../iotdb/consensus/simple/RecoveryTest.java       |    2 -
 .../Apache-IoTDB-ConfigNode-Dashboard.json         | 3310 +++++++++------
 .../Apache-IoTDB-DataNode-Dashboard.json           | 4285 +++++++++++++-------
 docs/UserGuide/Monitor-Alert/Metric-Tool.md        |   77 +-
 docs/zh/UserGuide/Monitor-Alert/Metric-Tool.md     |   60 +-
 .../iotdb/commons/service/metric/enums/Metric.java |    2 +
 .../db/consensus/DataRegionConsensusImpl.java      |    2 -
 .../db/consensus/SchemaRegionConsensusImpl.java    |    2 -
 21 files changed, 5114 insertions(+), 2978 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
index b49123c542..cca0c80433 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
@@ -20,7 +20,6 @@
 package org.apache.iotdb.confignode.manager.consensus;
 
 import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.consensus.ConfigRegionId;
@@ -91,7 +90,6 @@ public class ConsensusManager {
                       .setThisNode(
                           new TEndPoint(CONF.getInternalAddress(), CONF.getConsensusPort()))
                       .setStorageDir(CONF.getConsensusDir())
-                      .setConsensusGroupType(TConsensusGroupType.ConfigRegion)
                       .build(),
                   gid -> stateMachine)
               .orElseThrow(
@@ -180,7 +178,6 @@ public class ConsensusManager {
                                       .build())
                               .build())
                       .setStorageDir(CONF.getConsensusDir())
-                      .setConsensusGroupType(TConsensusGroupType.ConfigRegion)
                       .build(),
                   gid -> stateMachine)
               .orElseThrow(
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java b/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java
index 36aa9fa937..92db8f8a84 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/config/ConsensusConfig.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.consensus.config;
 
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 
 import java.util.Optional;
@@ -29,7 +28,6 @@ public class ConsensusConfig {
   private final TEndPoint thisNodeEndPoint;
   private final int thisNodeId;
   private final String storageDir;
-  private final TConsensusGroupType consensusGroupType;
   private final RatisConfig ratisConfig;
   private final IoTConsensusConfig ioTConsensusConfig;
 
@@ -37,13 +35,11 @@ public class ConsensusConfig {
       TEndPoint thisNode,
       int thisNodeId,
       String storageDir,
-      TConsensusGroupType consensusGroupType,
       RatisConfig ratisConfig,
       IoTConsensusConfig ioTConsensusConfig) {
     this.thisNodeEndPoint = thisNode;
     this.thisNodeId = thisNodeId;
     this.storageDir = storageDir;
-    this.consensusGroupType = consensusGroupType;
     this.ratisConfig = ratisConfig;
     this.ioTConsensusConfig = ioTConsensusConfig;
   }
@@ -60,10 +56,6 @@ public class ConsensusConfig {
     return storageDir;
   }
 
-  public TConsensusGroupType getConsensusGroupType() {
-    return consensusGroupType;
-  }
-
   public RatisConfig getRatisConfig() {
     return ratisConfig;
   }
@@ -81,7 +73,6 @@ public class ConsensusConfig {
     private TEndPoint thisNode;
     private int thisNodeId;
     private String storageDir;
-    private TConsensusGroupType consensusGroupType;
     private RatisConfig ratisConfig;
     private IoTConsensusConfig ioTConsensusConfig;
 
@@ -90,7 +81,6 @@ public class ConsensusConfig {
           thisNode,
           thisNodeId,
           storageDir,
-          consensusGroupType,
           Optional.ofNullable(ratisConfig).orElseGet(() -> RatisConfig.newBuilder().build()),
           Optional.ofNullable(ioTConsensusConfig)
               .orElseGet(() -> IoTConsensusConfig.newBuilder().build()));
@@ -111,11 +101,6 @@ public class ConsensusConfig {
       return this;
     }
 
-    public Builder setConsensusGroupType(TConsensusGroupType consensusGroupType) {
-      this.consensusGroupType = consensusGroupType;
-      return this;
-    }
-
     public Builder setRatisConfig(RatisConfig ratisConfig) {
       this.ratisConfig = ratisConfig;
       return this;
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 da94ded6b0..b256079623 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
@@ -27,6 +27,7 @@ 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;
@@ -60,6 +61,7 @@ public class ApplicationStateMachineProxy extends BaseStateMachine {
   private final IStateMachine.RetryPolicy retryPolicy;
   private final SnapshotStorage snapshotStorage;
   private final RaftGroupId groupId;
+  private final String consensusGroupType;
 
   public ApplicationStateMachineProxy(IStateMachine stateMachine, RaftGroupId id) {
     applicationStateMachine = stateMachine;
@@ -69,6 +71,7 @@ public class ApplicationStateMachineProxy extends BaseStateMachine {
             ? (IStateMachine.RetryPolicy) applicationStateMachine
             : new IStateMachine.RetryPolicy() {};
     snapshotStorage = new SnapshotStorage(applicationStateMachine, groupId);
+    consensusGroupType = Utils.getConsensusGroupTypeFromPrefix(groupId.toString());
     applicationStateMachine.start();
   }
 
@@ -109,6 +112,7 @@ public class ApplicationStateMachineProxy extends BaseStateMachine {
   @Override
   public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
     boolean isLeader = false;
+    long writeToStateMachineStartTime = System.nanoTime();
     RaftProtos.LogEntryProto log = trx.getLogEntry();
     updateLastAppliedTermIndex(log.getTerm(), log.getIndex());
 
@@ -140,18 +144,7 @@ public class ApplicationStateMachineProxy extends BaseStateMachine {
         IConsensusRequest deserializedRequest =
             applicationStateMachine.deserializeRequest(applicationRequest);
 
-        long startWriteTime = System.nanoTime();
         TSStatus result = applicationStateMachine.write(deserializedRequest);
-        if (isLeader) {
-          MetricService.getInstance()
-              .timer(
-                  System.nanoTime() - startWriteTime,
-                  TimeUnit.NANOSECONDS,
-                  Metric.PERFORMANCE_OVERVIEW_STORAGE_DETAIL.toString(),
-                  MetricLevel.IMPORTANT,
-                  Tag.STAGE.toString(),
-                  PerformanceOverviewMetrics.ENGINE);
-        }
 
         if (firstTry) {
           finalStatus = result;
@@ -179,7 +172,20 @@ public class ApplicationStateMachineProxy extends BaseStateMachine {
         }
       }
     } while (shouldRetry);
-
+    if (isLeader) {
+      MetricService.getInstance()
+          .timer(
+              System.nanoTime() - writeToStateMachineStartTime,
+              TimeUnit.NANOSECONDS,
+              Metric.PERFORMANCE_OVERVIEW_STORAGE_DETAIL.toString(),
+              MetricLevel.IMPORTANT,
+              Tag.STAGE.toString(),
+              PerformanceOverviewMetrics.ENGINE);
+      // statistic the time of write stateMachine
+      RatisMetricsManager.getInstance()
+          .recordWriteStateMachineCost(
+              System.nanoTime() - writeToStateMachineStartTime, consensusGroupType);
+    }
     return CompletableFuture.completedFuture(ret);
   }
 
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
index c80ae31fb1..38242b56d4 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
@@ -50,6 +50,7 @@ import org.apache.iotdb.consensus.exception.PeerAlreadyInConsensusGroupException
 import org.apache.iotdb.consensus.exception.PeerNotInConsensusGroupException;
 import org.apache.iotdb.consensus.exception.RatisRequestFailedException;
 import org.apache.iotdb.consensus.ratis.metrics.RatisMetricSet;
+import org.apache.iotdb.consensus.ratis.metrics.RatisMetricsManager;
 
 import org.apache.commons.pool2.KeyedObjectPool;
 import org.apache.commons.pool2.impl.GenericKeyedObjectPool;
@@ -125,6 +126,8 @@ class RatisConsensus implements IConsensus {
 
   private final RatisMetricSet ratisMetricSet;
 
+  private String consensusGroupType = null;
+
   public RatisConsensus(ConsensusConfig config, IStateMachine.Registry registry)
       throws IOException {
     myself =
@@ -141,8 +144,7 @@ class RatisConsensus implements IConsensus {
 
     Utils.initRatisConfig(properties, config.getRatisConfig());
     this.config = config.getRatisConfig();
-
-    this.ratisMetricSet = new RatisMetricSet(config.getConsensusGroupType());
+    this.ratisMetricSet = new RatisMetricSet();
 
     clientManager =
         new IClientManager.Factory<RaftGroup, RatisClient>()
@@ -235,7 +237,6 @@ class RatisConsensus implements IConsensus {
   @Override
   public ConsensusWriteResponse write(
       ConsensusGroupId consensusGroupId, IConsensusRequest IConsensusRequest) {
-
     // pre-condition: group exists and myself server serves this group
     RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(consensusGroupId);
     RaftGroup raftGroup = getGroupInfo(raftGroupId);
@@ -259,6 +260,8 @@ class RatisConsensus implements IConsensus {
     // 1. first try the local server
     RaftClientRequest clientRequest =
         buildRawRequest(raftGroupId, message, RaftClientRequest.writeRequestType());
+
+    long writeToRatisStartTime = System.nanoTime();
     RaftClientReply localServerReply;
     RaftPeer suggestedLeader = null;
     if (isLeader(consensusGroupId) && waitUntilLeaderReady(raftGroupId)) {
@@ -275,6 +278,13 @@ class RatisConsensus implements IConsensus {
         }
       } catch (IOException e) {
         return failedWrite(new RatisRequestFailedException(e));
+      } finally {
+        if (consensusGroupType == null) {
+          consensusGroupType = Utils.getConsensusGroupTypeFromPrefix(raftGroupId.toString());
+        }
+        // statistic the time of write locally
+        RatisMetricsManager.getInstance()
+            .recordWriteLocallyCost(System.nanoTime() - writeToRatisStartTime, consensusGroupType);
       }
     }
 
@@ -294,13 +304,18 @@ class RatisConsensus implements IConsensus {
       if (client != null) {
         client.returnSelf();
       }
+      if (consensusGroupType == null) {
+        consensusGroupType = Utils.getConsensusGroupTypeFromPrefix(raftGroupId.toString());
+      }
+      // statistic the time of write remotely
+      RatisMetricsManager.getInstance()
+          .recordWriteRemotelyCost(System.nanoTime() - writeToRatisStartTime, consensusGroupType);
     }
 
     if (suggestedLeader != null) {
       TEndPoint leaderEndPoint = Utils.fromRaftPeerAddressToTEndPoint(suggestedLeader.getAddress());
       writeResult.setRedirectNode(new TEndPoint(leaderEndPoint.getIp(), leaderEndPoint.getPort()));
     }
-
     return ConsensusWriteResponse.newBuilder().setStatus(writeResult).build();
   }
 
@@ -319,7 +334,14 @@ class RatisConsensus implements IConsensus {
       RequestMessage message = new RequestMessage(IConsensusRequest);
       RaftClientRequest clientRequest =
           buildRawRequest(groupId, message, RaftClientRequest.staleReadRequestType(-1));
+      long readRatisStartTime = System.nanoTime();
       reply = server.submitClientRequest(clientRequest);
+      if (consensusGroupType == null) {
+        consensusGroupType = Utils.getConsensusGroupTypeFromPrefix(groupId.toString());
+      }
+      // statistic the time of submit read request
+      RatisMetricsManager.getInstance()
+          .recordReadRequestCost(System.nanoTime() - readRatisStartTime, consensusGroupType);
       if (!reply.isSuccess()) {
         return failedRead(new RatisRequestFailedException(reply.getException()));
       }
@@ -330,7 +352,6 @@ class RatisConsensus implements IConsensus {
     Message ret = reply.getMessage();
     ResponseMessage readResponseMessage = (ResponseMessage) ret;
     DataSet dataSet = (DataSet) readResponseMessage.getContentHolder();
-
     return ConsensusReadResponse.newBuilder().setDataSet(dataSet).build();
   }
 
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
index 941dfc76f8..7d7954a530 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/Utils.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.consensus.ratis;
 
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
@@ -47,6 +48,8 @@ import java.util.stream.Collectors;
 public class Utils {
   private static final int TEMP_BUFFER_SIZE = 1024;
   private static final byte PADDING_MAGIC = 0x47;
+  private static final String DATA_REGION_GROUP = "group-0001";
+  private static final String SCHEMA_REGION_GROUP = "group-0002";
 
   private Utils() {}
 
@@ -174,6 +177,18 @@ public class Utils {
     return TermIndex.valueOf(Long.parseLong(items[0]), Long.parseLong(items[1]));
   }
 
+  public static String getConsensusGroupTypeFromPrefix(String prefix) {
+    TConsensusGroupType consensusGroupType;
+    if (prefix.contains(DATA_REGION_GROUP)) {
+      consensusGroupType = TConsensusGroupType.DataRegion;
+    } else if (prefix.contains(SCHEMA_REGION_GROUP)) {
+      consensusGroupType = TConsensusGroupType.SchemaRegion;
+    } else {
+      consensusGroupType = TConsensusGroupType.ConfigRegion;
+    }
+    return consensusGroupType.toString();
+  }
+
   public static void initRatisConfig(RaftProperties properties, RatisConfig config) {
     GrpcConfigKeys.setMessageSizeMax(properties, config.getGrpc().getMessageSizeMax());
     GrpcConfigKeys.setFlowControlWindow(properties, config.getGrpc().getFlowControlWindow());
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java
index 8b9af65e6f..7846c61b89 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.consensus.ratis.metrics;
 
+import org.apache.iotdb.consensus.ratis.Utils;
 import org.apache.iotdb.metrics.AbstractMetricService;
 import org.apache.iotdb.metrics.utils.MetricLevel;
 import org.apache.iotdb.metrics.utils.MetricType;
@@ -36,6 +37,8 @@ import com.codahale.metrics.Timer;
 import org.apache.ratis.metrics.MetricRegistryInfo;
 import org.apache.ratis.metrics.RatisMetricRegistry;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.concurrent.ConcurrentHashMap;
@@ -49,28 +52,72 @@ public class IoTDBMetricRegistry implements RatisMetricRegistry {
   private final Map<String, CounterProxy> counterCache = new ConcurrentHashMap<>();
   private final Map<String, TimerProxy> timerCache = new ConcurrentHashMap<>();
   private final Map<String, GaugeProxy> gaugeCache = new ConcurrentHashMap<>();
-
-  IoTDBMetricRegistry(
-      MetricRegistryInfo info, AbstractMetricService service, String consensusGroupType) {
+  /** Time taken to flush log. */
+  public static final String RAFT_LOG_FLUSH_TIME = "flushTime";
+  /** Size of SegmentedRaftLogCache::closedSegments in bytes */
+  public static final String RAFT_LOG_CACHE_CLOSED_SEGMENTS_SIZE_IN_BYTES =
+      "closedSegmentsSizeInBytes";
+  /** Size of SegmentedRaftLogCache::openSegment in bytes */
+  public static final String RAFT_LOG_CACHE_OPEN_SEGMENT_SIZE_IN_BYTES = "openSegmentSizeInBytes";
+  /** Total time taken to append a raft log entry */
+  public static final String RAFT_LOG_APPEND_ENTRY_LATENCY = "appendEntryLatency";
+  /**
+   * Time taken for a Raft log operation to get into the queue after being requested. This is the
+   * time that it has to wait for the queue to be non-full.
+   */
+  public static final String RAFT_LOG_TASK_ENQUEUE_DELAY = "queueingDelay";
+  /** Time spent by a Raft log operation in the queue. */
+  public static final String RAFT_LOG_TASK_QUEUE_TIME = "enqueuedTime";
+  /** Time taken for a Raft log operation to complete execution. */
+  public static final String RAFT_LOG_TASK_EXECUTION_TIME = "ExecutionTime";
+  /** Time taken for followers to append log entries. */
+  public static final String FOLLOWER_APPEND_ENTRIES_LATENCY = "follower_append_entry_latency";
+  /** Time taken to process write requests from client. */
+  public static final String RAFT_CLIENT_WRITE_REQUEST = "clientWriteRequest";
+
+  private static final List<String> RATIS_METRICS = new ArrayList<>();
+
+  static {
+    RATIS_METRICS.add(RAFT_LOG_FLUSH_TIME);
+    RATIS_METRICS.add(RAFT_LOG_CACHE_CLOSED_SEGMENTS_SIZE_IN_BYTES);
+    RATIS_METRICS.add(RAFT_LOG_CACHE_OPEN_SEGMENT_SIZE_IN_BYTES);
+    RATIS_METRICS.add(RAFT_LOG_APPEND_ENTRY_LATENCY);
+    RATIS_METRICS.add(RAFT_LOG_TASK_ENQUEUE_DELAY);
+    RATIS_METRICS.add(RAFT_LOG_TASK_QUEUE_TIME);
+    RATIS_METRICS.add(RAFT_LOG_TASK_EXECUTION_TIME);
+    RATIS_METRICS.add(FOLLOWER_APPEND_ENTRIES_LATENCY);
+    RATIS_METRICS.add(RAFT_CLIENT_WRITE_REQUEST);
+  }
+
+  IoTDBMetricRegistry(MetricRegistryInfo info, AbstractMetricService service) {
     this.info = info;
     this.metricService = service;
     prefix =
         MetricRegistry.name(
-            consensusGroupType,
+            Utils.getConsensusGroupTypeFromPrefix(info.getPrefix()),
             info.getApplicationName(),
-            info.getMetricsComponentName(),
-            info.getPrefix());
+            info.getMetricsComponentName());
   }
 
   private String getMetricName(String name) {
     return metricNameCache.computeIfAbsent(name, n -> MetricRegistry.name(prefix, n));
   }
 
+  public MetricLevel getMetricLevel(String name) {
+    for (String ratisMetric : RATIS_METRICS) {
+      if (name.contains(ratisMetric)) {
+        return MetricLevel.IMPORTANT;
+      }
+    }
+    return MetricLevel.CORE;
+  }
+
   @Override
   public Timer timer(String name) {
     final String fullName = getMetricName(name);
     return timerCache.computeIfAbsent(
-        fullName, fn -> new TimerProxy(metricService.getOrCreateTimer(fn, MetricLevel.IMPORTANT)));
+        fullName,
+        fn -> new TimerProxy(metricService.getOrCreateTimer(fn, getMetricLevel(fullName))));
   }
 
   @Override
@@ -80,7 +127,7 @@ public class IoTDBMetricRegistry implements RatisMetricRegistry {
         fullName,
         fn ->
             new CounterProxy(
-                metricService.getOrCreateCounter(getMetricName(name), MetricLevel.IMPORTANT)));
+                metricService.getOrCreateCounter(getMetricName(name), getMetricLevel(fullName))));
   }
 
   @Override
@@ -124,7 +171,7 @@ public class IoTDBMetricRegistry implements RatisMetricRegistry {
         gaugeName -> {
           final GaugeProxy gauge = new GaugeProxy(metricSupplier);
           metricService.createAutoGauge(
-              gaugeName, MetricLevel.IMPORTANT, gauge, GaugeProxy::getValueAsDouble);
+              gaugeName, getMetricLevel(fullName), gauge, GaugeProxy::getValueAsDouble);
           return gauge;
         });
   }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java
index 898ebd4680..de62b45939 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java
@@ -18,8 +18,8 @@
  */
 package org.apache.iotdb.consensus.ratis.metrics;
 
+import org.apache.iotdb.commons.service.metric.MetricService;
 import org.apache.iotdb.metrics.AbstractMetricService;
-import org.apache.iotdb.metrics.DoNothingMetricService;
 
 import org.apache.ratis.metrics.MetricRegistries;
 import org.apache.ratis.metrics.MetricRegistryInfo;
@@ -36,9 +36,7 @@ public class MetricRegistryManager extends MetricRegistries {
   /** Using RefCountingMap here because of potential duplicate MetricRegistryInfos */
   private final RefCountingMap<MetricRegistryInfo, RatisMetricRegistry> registries;
   /** TODO: enable ratis metrics after verifying its correctness and efficiency */
-  private final AbstractMetricService service = new DoNothingMetricService();
-
-  private String consensusGroupType;
+  private final AbstractMetricService service = MetricService.getInstance();
 
   public MetricRegistryManager() {
     this.registries = new RefCountingMap<>();
@@ -55,8 +53,7 @@ public class MetricRegistryManager extends MetricRegistries {
   @Override
   public RatisMetricRegistry create(MetricRegistryInfo metricRegistryInfo) {
     return registries.put(
-        metricRegistryInfo,
-        () -> new IoTDBMetricRegistry(metricRegistryInfo, service, consensusGroupType));
+        metricRegistryInfo, () -> new IoTDBMetricRegistry(metricRegistryInfo, service));
   }
 
   @Override
@@ -98,8 +95,4 @@ public class MetricRegistryManager extends MetricRegistries {
     throw new UnsupportedOperationException(
         "Console Reporter is disabled from RatisMetricRegistries");
   }
-
-  public void setConsensusGroupType(String consensusGroupType) {
-    this.consensusGroupType = consensusGroupType;
-  }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java
index 732f4015d4..7e20c94466 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java
@@ -19,30 +19,97 @@
 package org.apache.iotdb.consensus.ratis.metrics;
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.commons.service.metric.enums.Metric;
+import org.apache.iotdb.commons.service.metric.enums.Tag;
 import org.apache.iotdb.metrics.AbstractMetricService;
 import org.apache.iotdb.metrics.metricsets.IMetricSet;
+import org.apache.iotdb.metrics.utils.MetricInfo;
+import org.apache.iotdb.metrics.utils.MetricLevel;
+import org.apache.iotdb.metrics.utils.MetricType;
 
 import org.apache.ratis.metrics.MetricRegistries;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 public class RatisMetricSet implements IMetricSet {
   private MetricRegistries manager;
-  private final String consensusGroupType;
+  private static final Map<String, MetricInfo> metricInfoMap = new HashMap<>();
+  private static final String RATIS_CONSENSUS_WRITE = Metric.RATIS_CONSENSUS_WRITE.toString();
+  private static final String RATIS_CONSENSUS_READ = Metric.RATIS_CONSENSUS_READ.toString();
+
+  private static final String DATA_REGION_RATIS_CONSENSUS_WRITE =
+      TConsensusGroupType.DataRegion + "_" + RATIS_CONSENSUS_WRITE;
+  private static final String DATA_REGION_RATIS_CONSENSUS_READ =
+      TConsensusGroupType.DataRegion + "_" + RATIS_CONSENSUS_READ;
+  private static final String SCHEMA_REGION_RATIS_CONSENSUS_WRITE =
+      TConsensusGroupType.SchemaRegion + "_" + RATIS_CONSENSUS_WRITE;
+  private static final String SCHEMA_REGION_RATIS_CONSENSUS_READ =
+      TConsensusGroupType.SchemaRegion + "_" + RATIS_CONSENSUS_READ;
+  private static final String CONFIG_REGION_RATIS_CONSENSUS_WRITE =
+      TConsensusGroupType.ConfigRegion + "_" + RATIS_CONSENSUS_WRITE;
+  private static final String CONFIG_REGION_RATIS_CONSENSUS_READ =
+      TConsensusGroupType.ConfigRegion + "_" + RATIS_CONSENSUS_READ;
+
+  public static final String WRITE_LOCALLY = "writeLocally";
+  public static final String WRITE_REMOTELY = "writeRemotely";
+  public static final String SUBMIT_READ_REQUEST = "submitReadRequest";
+  public static final String WRITE_STATE_MACHINE = "writeStateMachine";
+  private static final List<String> RATIS_WRITE_METRICS = new ArrayList<>();
+  private static final List<String> RATIS_WRITE_STAGES = new ArrayList<>();
+  private static final List<String> RATIS_READ_METRICS = new ArrayList<>();
+  private static final List<String> RATIS_READ_STAGES = new ArrayList<>();
+
+  static {
+    RATIS_WRITE_METRICS.add(DATA_REGION_RATIS_CONSENSUS_WRITE);
+    RATIS_WRITE_METRICS.add(SCHEMA_REGION_RATIS_CONSENSUS_WRITE);
+    RATIS_WRITE_METRICS.add(CONFIG_REGION_RATIS_CONSENSUS_WRITE);
+
+    RATIS_READ_METRICS.add(DATA_REGION_RATIS_CONSENSUS_READ);
+    RATIS_READ_METRICS.add(SCHEMA_REGION_RATIS_CONSENSUS_READ);
+    RATIS_READ_METRICS.add(CONFIG_REGION_RATIS_CONSENSUS_READ);
+
+    RATIS_WRITE_STAGES.add(WRITE_LOCALLY);
+    RATIS_WRITE_STAGES.add(WRITE_REMOTELY);
+    RATIS_WRITE_STAGES.add(WRITE_STATE_MACHINE);
 
-  public RatisMetricSet(TConsensusGroupType consensusGroupType) {
-    super();
-    this.consensusGroupType = consensusGroupType.toString();
+    RATIS_READ_STAGES.add(SUBMIT_READ_REQUEST);
+
+    for (String ratisWriteMetric : RATIS_WRITE_METRICS) {
+      for (String ratisWriteStage : RATIS_WRITE_STAGES) {
+        metricInfoMap.put(
+            ratisWriteStage,
+            new MetricInfo(
+                MetricType.TIMER, ratisWriteMetric, Tag.STAGE.toString(), ratisWriteStage));
+      }
+    }
+
+    for (String ratisReadMetric : RATIS_READ_METRICS) {
+      for (String ratisReadStage : RATIS_READ_STAGES) {
+        metricInfoMap.put(
+            ratisReadStage,
+            new MetricInfo(
+                MetricType.TIMER, ratisReadMetric, Tag.STAGE.toString(), ratisReadStage));
+      }
+    }
   }
 
   @Override
   public void bindTo(AbstractMetricService metricService) {
     manager = MetricRegistries.global();
-    if (manager instanceof MetricRegistryManager) {
-      ((MetricRegistryManager) manager).setConsensusGroupType(consensusGroupType);
+    for (MetricInfo metricInfo : metricInfoMap.values()) {
+      metricService.getOrCreateTimer(
+          metricInfo.getName(), MetricLevel.CORE, metricInfo.getTagsInArray());
     }
   }
 
   @Override
   public void unbindFrom(AbstractMetricService metricService) {
     manager.clear();
+    for (MetricInfo metricInfo : metricInfoMap.values()) {
+      metricService.remove(MetricType.TIMER, metricInfo.getName(), metricInfo.getTagsInArray());
+    }
   }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricsManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricsManager.java
new file mode 100644
index 0000000000..b7d11a732f
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricsManager.java
@@ -0,0 +1,84 @@
+/*
+ * 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.consensus.ratis.metrics;
+
+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.Tag;
+import org.apache.iotdb.metrics.utils.MetricLevel;
+
+import java.util.concurrent.TimeUnit;
+
+public class RatisMetricsManager {
+  private final MetricService metricService = MetricService.getInstance();
+  /** Record the time cost in write locally stage. */
+  public void recordWriteLocallyCost(long costTimeInNanos, String consensusGroupType) {
+    metricService.timer(
+        costTimeInNanos,
+        TimeUnit.NANOSECONDS,
+        consensusGroupType + "_" + Metric.RATIS_CONSENSUS_WRITE,
+        MetricLevel.IMPORTANT,
+        Tag.STAGE.toString(),
+        RatisMetricSet.WRITE_LOCALLY);
+  }
+
+  /** Record the time cost in write remotely stage. */
+  public void recordWriteRemotelyCost(long costTimeInNanos, String consensusGroupType) {
+    metricService.timer(
+        costTimeInNanos,
+        TimeUnit.NANOSECONDS,
+        consensusGroupType + "_" + Metric.RATIS_CONSENSUS_WRITE,
+        MetricLevel.IMPORTANT,
+        Tag.STAGE.toString(),
+        RatisMetricSet.WRITE_REMOTELY);
+  }
+  /** Record the time cost in submit read request stage. */
+  public void recordReadRequestCost(long costTimeInNanos, String consensusGroupType) {
+    metricService.timer(
+        costTimeInNanos,
+        TimeUnit.NANOSECONDS,
+        consensusGroupType + "_" + Metric.RATIS_CONSENSUS_READ,
+        MetricLevel.IMPORTANT,
+        Tag.STAGE.toString(),
+        RatisMetricSet.SUBMIT_READ_REQUEST);
+  }
+
+  /** Record the time cost in write state machine stage. */
+  public void recordWriteStateMachineCost(long costTimeInNanos, String consensusGroupType) {
+    metricService.timer(
+        costTimeInNanos,
+        TimeUnit.NANOSECONDS,
+        consensusGroupType + "_" + Metric.RATIS_CONSENSUS_WRITE,
+        MetricLevel.IMPORTANT,
+        Tag.STAGE.toString(),
+        RatisMetricSet.WRITE_STATE_MACHINE);
+  }
+
+  public static RatisMetricsManager getInstance() {
+    return RatisMetricsManagerHolder.INSTANCE;
+  }
+
+  private static class RatisMetricsManagerHolder {
+    private static final RatisMetricsManager INSTANCE = new RatisMetricsManager();
+
+    private RatisMetricsManagerHolder() {
+      // empty constructor
+    }
+  }
+}
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/iot/ReplicateTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/iot/ReplicateTest.java
index 0c9321fc2b..a36db6e3ea 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/iot/ReplicateTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/iot/ReplicateTest.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.consensus.iot;
 
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
@@ -94,7 +93,6 @@ public class ReplicateTest {
                           .setThisNodeId(peers.get(i).getNodeId())
                           .setThisNode(peers.get(i).getEndpoint())
                           .setStorageDir(peersStorage.get(i).getAbsolutePath())
-                          .setConsensusGroupType(TConsensusGroupType.DataRegion)
                           .build(),
                       groupId -> stateMachines.get(finalI))
                   .orElseThrow(
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/iot/StabilityTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/iot/StabilityTest.java
index 5f04af6ee0..ea7c15c8f1 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/iot/StabilityTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/iot/StabilityTest.java
@@ -19,7 +19,6 @@
 
 package org.apache.iotdb.consensus.iot;
 
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
@@ -57,7 +56,6 @@ public class StabilityTest {
                     .setThisNodeId(1)
                     .setThisNode(new TEndPoint("0.0.0.0", 9000))
                     .setStorageDir(storageDir.getAbsolutePath())
-                    .setConsensusGroupType(TConsensusGroupType.DataRegion)
                     .build(),
                 gid -> new TestStateMachine())
             .orElseThrow(
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
index 6a5bbde7da..cb86a0f79d 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RatisConsensusTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.consensus.ratis;
 
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.DataRegionId;
@@ -100,7 +99,6 @@ public class RatisConsensusTest {
                       .setThisNode(peers.get(i).getEndpoint())
                       .setRatisConfig(config)
                       .setStorageDir(peersStorage.get(i).getAbsolutePath())
-                      .setConsensusGroupType(TConsensusGroupType.DataRegion)
                       .build(),
                   groupId -> stateMachines.get(finalI))
               .orElseThrow(
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/SnapshotTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/SnapshotTest.java
index 5ce5d83fe0..9a642f1bda 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/ratis/SnapshotTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/ratis/SnapshotTest.java
@@ -18,6 +18,9 @@
  */
 package org.apache.iotdb.consensus.ratis;
 
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
+
+import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.storage.RaftStorage;
 import org.apache.ratis.server.storage.RaftStorageDirectory;
@@ -94,8 +97,10 @@ public class SnapshotTest {
 
   @Test
   public void testSnapshot() throws Exception {
+    ConsensusGroupId consensusGroupId = ConsensusGroupId.Factory.create(0, 0);
+    RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(consensusGroupId);
     ApplicationStateMachineProxy proxy =
-        new ApplicationStateMachineProxy(new TestUtils.IntegerCounter(), null);
+        new ApplicationStateMachineProxy(new TestUtils.IntegerCounter(), raftGroupId);
 
     proxy.initialize(null, null, new EmptyStorageWithOnlySMDir());
 
@@ -166,8 +171,10 @@ public class SnapshotTest {
 
   @Test
   public void testCrossDiskLinkSnapshot() throws Exception {
+    ConsensusGroupId consensusGroupId = ConsensusGroupId.Factory.create(0, 0);
+    RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(consensusGroupId);
     ApplicationStateMachineProxy proxy =
-        new ApplicationStateMachineProxy(new CrossDiskLinkStatemachine(), null);
+        new ApplicationStateMachineProxy(new CrossDiskLinkStatemachine(), raftGroupId);
 
     proxy.initialize(null, null, new EmptyStorageWithOnlySMDir());
     proxy.notifyTermIndexUpdated(20, 1005);
diff --git a/consensus/src/test/java/org/apache/iotdb/consensus/simple/RecoveryTest.java b/consensus/src/test/java/org/apache/iotdb/consensus/simple/RecoveryTest.java
index 496f24e0a7..83fc2414b9 100644
--- a/consensus/src/test/java/org/apache/iotdb/consensus/simple/RecoveryTest.java
+++ b/consensus/src/test/java/org/apache/iotdb/consensus/simple/RecoveryTest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.consensus.simple;
 
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
@@ -52,7 +51,6 @@ public class RecoveryTest {
                     .setThisNodeId(1)
                     .setThisNode(new TEndPoint("0.0.0.0", 9000))
                     .setStorageDir("target" + java.io.File.separator + "recovery")
-                    .setConsensusGroupType(TConsensusGroupType.SchemaRegion)
                     .build(),
                 gid -> new EmptyStateMachine())
             .orElseThrow(
diff --git a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
index aee87c1316..7016071f59 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
@@ -75,12 +75,12 @@
   "fiscalYearStartMonth": 0,
   "graphTooltip": 1,
   "id": null,
-  "iteration": 1678081811954,
+  "iteration": 1678983654736,
   "links": [],
   "liveNow": false,
   "panels": [
     {
-      "collapsed": false,
+      "collapsed": true,
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -88,1119 +88,1120 @@
         "y": 0
       },
       "id": 57,
-      "panels": [],
-      "title": "Overview",
-      "type": "row"
-    },
-    {
-      "description": "The current status of cluster ConfigNodes",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            }
-          },
-          "mappings": []
-        },
-        "overrides": [
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"ReadOnly\", type=\"ConfigNode\"}"
-            },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "orange",
-                  "mode": "fixed"
+      "panels": [
+        {
+          "description": "The current status of cluster ConfigNodes",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
                 }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Removing\", type=\"ConfigNode\"}"
+              },
+              "mappings": []
             },
-            "properties": [
+            "overrides": [
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "red",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Unknown\", type=\"ConfigNode\"}"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"ReadOnly\", type=\"ConfigNode\"}"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "yellow",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Running\", type=\"ConfigNode\"}"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Removing\", type=\"ConfigNode\"}"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "green",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Running"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Unknown\", type=\"ConfigNode\"}"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "green",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "ReadOnly"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Running\", type=\"ConfigNode\"}"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "orange",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Removing"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "Running"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "red",
-                  "mode": "fixed"
-                }
+                "matcher": {
+                  "id": "byName",
+                  "options": "ReadOnly"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Removing"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Unknown"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               }
             ]
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Unknown"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 0,
+            "y": 1
+          },
+          "id": 59,
+          "options": {
+            "legend": {
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "values": [
+                "value"
+              ]
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "yellow",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          }
-        ]
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 0,
-        "y": 1
-      },
-      "id": 59,
-      "options": {
-        "legend": {
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "values": [
-            "value"
-          ]
-        },
-        "pieType": "pie",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+            "pieType": "pie",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "node_num{instance=\"$instance\", type=\"ConfigNode\"}",
+              "interval": "",
+              "legendFormat": "{{status}}",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "ConfigNode Current Status",
+          "type": "piechart"
         },
-        "tooltip": {
-          "mode": "single",
-          "sort": "none"
-        }
-      },
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "node_num{instance=\"$instance\", type=\"ConfigNode\"}",
-          "interval": "",
-          "legendFormat": "{{status}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "ConfigNode Current Status",
-      "type": "piechart"
-    },
-    {
-      "description": "The number of cluster Databases",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
-          },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
+          "description": "The number of cluster Databases",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
               }
-            ]
-          }
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 4,
-        "y": 1
-      },
-      "id": 79,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "area",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
-        },
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            },
+            "overrides": []
           },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "database_num{instance=\"$instance\"}",
-          "interval": "",
-          "legendFormat": "__auto",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "Database Count",
-      "type": "stat"
-    },
-    {
-      "description": "The number of cluster DataRegions",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 4,
+            "y": 1
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
-          }
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 8,
-        "y": 1
-      },
-      "id": 85,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "area",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "id": 79,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "area",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
+          },
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "database_num{instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "__auto",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "Database Count",
+          "type": "stat"
         },
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "description": "The number of cluster DataRegions",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              }
+            },
+            "overrides": []
           },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "sum by (type) (region_num{type=\"DataRegion\", instance=\"$instance\"})",
-          "interval": "",
-          "legendFormat": "__auto",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "DataRegion Count",
-      "type": "stat"
-    },
-    {
-      "description": "The number of cluster SchemaRegions",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 8,
+            "y": 1
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
-          }
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 12,
-        "y": 1
-      },
-      "id": 83,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "area",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "id": 85,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "area",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
+          },
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "sum by (type) (region_num{type=\"DataRegion\", instance=\"$instance\"})",
+              "interval": "",
+              "legendFormat": "__auto",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "DataRegion Count",
+          "type": "stat"
         },
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "description": "The number of cluster SchemaRegions",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              }
+            },
+            "overrides": []
           },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "sum by(type) (region_num{type=\"SchemaRegion\", instance=\"$instance\"})",
-          "interval": "",
-          "legendFormat": "__auto",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "SchemaRegion Count",
-      "type": "stat"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "description": "The CPU Core of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 12,
+            "y": 1
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
+          "id": 83,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "area",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
           },
-          "unit": "Core(s)"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 16,
-        "y": 1
-      },
-      "id": 69,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "sum by(type) (region_num{type=\"SchemaRegion\", instance=\"$instance\"})",
+              "interval": "",
+              "legendFormat": "__auto",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "SchemaRegion Count",
+          "type": "stat"
         },
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "code",
-          "exemplar": false,
-          "expr": "sys_cpu_cores{instance=\"$instance\", name=\"system\"}",
-          "instant": false,
-          "interval": "",
-          "legendFormat": "{{name}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "CPU Core",
-      "type": "stat"
-    },
-    {
-      "description": "The total disk space of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
-          },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
+          "description": "The CPU Core of current ConfigNode",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              },
+              "unit": "Core(s)"
+            },
+            "overrides": []
           },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 20,
-        "y": 1
-      },
-      "id": 77,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
-        },
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 16,
+            "y": 1
           },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "sys_disk_total_space{instance=\"$instance\"}",
-          "interval": "",
-          "legendFormat": "{{name}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "Total Disk Space",
-      "type": "stat"
-    },
-    {
-      "description": "The current status of cluster DataNodes",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
+          "id": 69,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
           },
-          "custom": {
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": false,
+              "expr": "sys_cpu_cores{instance=\"$instance\", name=\"system\"}",
+              "instant": false,
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
             }
-          },
-          "mappings": []
+          ],
+          "title": "CPU Core",
+          "type": "stat"
         },
-        "overrides": [
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Running"
+        {
+          "description": "The total disk space of current ConfigNode",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              },
+              "unit": "bytes"
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "green",
-                  "mode": "fixed"
-                }
-              }
-            ]
+            "overrides": []
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "ReadOnly"
-            },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "orange",
-                  "mode": "fixed"
-                }
-              }
-            ]
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 20,
+            "y": 1
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Removing"
+          "id": 77,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "red",
-                  "mode": "fixed"
-                }
-              }
-            ]
+            "textMode": "auto"
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Unknown"
-            },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "yellow",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          }
-        ]
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 0,
-        "y": 7
-      },
-      "id": 65,
-      "options": {
-        "legend": {
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "values": [
-            "value"
-          ]
-        },
-        "pieType": "pie",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "sys_disk_total_space{instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "Total Disk Space",
+          "type": "stat"
         },
-        "tooltip": {
-          "mode": "single",
-          "sort": "none"
-        }
-      },
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "node_num{instance=\"$instance\", type=\"DataNode\"}",
-          "interval": "",
-          "legendFormat": "{{status}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "DataNode Current Status",
-      "type": "piechart"
-    },
-    {
-      "description": "Bar of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
-          },
-          "mappings": [],
-          "max": 1,
-          "min": 0,
-          "thresholds": {
-            "mode": "percentage",
-            "steps": [
+          "description": "The current status of cluster DataNodes",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                }
+              },
+              "mappings": []
+            },
+            "overrides": [
               {
-                "color": "green",
-                "value": null
+                "matcher": {
+                  "id": "byName",
+                  "options": "Running"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               },
               {
-                "color": "#EAB839",
-                "value": 80
+                "matcher": {
+                  "id": "byName",
+                  "options": "ReadOnly"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               },
               {
-                "color": "red",
-                "value": 90
-              }
-            ]
-          },
-          "unit": "percentunit"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 4,
-        "y": 7
-      },
-      "id": 73,
-      "options": {
-        "displayMode": "lcd",
-        "minVizHeight": 10,
-        "minVizWidth": 0,
-        "orientation": "horizontal",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
-        },
-        "showUnfilled": true
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "code",
-          "expr": "process_cpu_load{instance=~\"$instance\"} / 100",
-          "hide": false,
-          "legendFormat": "cpu load",
-          "range": true,
-          "refId": "C"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "code",
-          "exemplar": true,
-          "expr": "1- sys_disk_free_space{instance=\"$instance\"} / sys_disk_total_space{instance=\"$instance\"}",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "disk",
-          "range": true,
-          "refId": "D"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "code",
-          "exemplar": true,
-          "expr": "1 - process_free_mem{instance=\"$instance\"} / process_total_mem{instance=\"$instance\"}",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "process memory",
-          "range": true,
-          "refId": "B"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "code",
-          "exemplar": false,
-          "expr": "1 - sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"} / sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
-          "instant": false,
-          "interval": "",
-          "legendFormat": "system memory",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "type": "bargauge"
-    },
-    {
-      "description": "The current status of cluster DataRegions",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
-          },
-          "custom": {
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            }
-          },
-          "mappings": []
-        },
-        "overrides": [
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Running"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "Removing"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "green",
-                  "mode": "fixed"
-                }
+                "matcher": {
+                  "id": "byName",
+                  "options": "Unknown"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               }
             ]
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "ReadOnly"
-            },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "orange",
-                  "mode": "fixed"
-                }
-              }
-            ]
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 0,
+            "y": 7
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Removing"
+          "id": 65,
+          "options": {
+            "legend": {
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "values": [
+                "value"
+              ]
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "red",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Unknown"
+            "pieType": "pie",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "yellow",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          }
-        ]
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 8,
-        "y": 7
-      },
-      "id": 93,
-      "options": {
-        "legend": {
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "values": [
-            "value"
-          ]
-        },
-        "pieType": "pie",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "node_num{instance=\"$instance\", type=\"DataNode\"}",
+              "interval": "",
+              "legendFormat": "{{status}}",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "DataNode Current Status",
+          "type": "piechart"
         },
-        "tooltip": {
-          "mode": "single",
-          "sort": "none"
-        }
-      },
-      "pluginVersion": "9.3.6",
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "description": "Bar of current ConfigNode",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "max": 1,
+              "min": 0,
+              "thresholds": {
+                "mode": "percentage",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "#EAB839",
+                    "value": 80
+                  },
+                  {
+                    "color": "red",
+                    "value": 90
+                  }
+                ]
+              },
+              "unit": "percentunit"
+            },
+            "overrides": []
           },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "region_num{type=\"DataRegion\", instance=\"$instance\"}",
-          "interval": "",
-          "legendFormat": "{{status}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "DataRegion Current Status",
-      "type": "piechart"
-    },
-    {
-      "description": "The current status of cluster SchemaRegions",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 4,
+            "y": 7
           },
-          "custom": {
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            }
+          "id": 73,
+          "options": {
+            "displayMode": "lcd",
+            "minVizHeight": 10,
+            "minVizWidth": 0,
+            "orientation": "horizontal",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "showUnfilled": true
           },
-          "mappings": []
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "process_cpu_load{instance=~\"$instance\"} / 100",
+              "hide": false,
+              "legendFormat": "cpu load",
+              "range": true,
+              "refId": "C"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "1- sys_disk_free_space{instance=\"$instance\"} / sys_disk_total_space{instance=\"$instance\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "disk",
+              "range": true,
+              "refId": "D"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "1 - process_free_mem{instance=\"$instance\"} / process_total_mem{instance=\"$instance\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "process memory",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": false,
+              "expr": "1 - sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"} / sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+              "instant": false,
+              "interval": "",
+              "legendFormat": "system memory",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "type": "bargauge"
         },
-        "overrides": [
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Running"
+        {
+          "description": "The current status of cluster DataRegions",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                }
+              },
+              "mappings": []
             },
-            "properties": [
+            "overrides": [
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Running"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "ReadOnly"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Removing"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "green",
-                  "mode": "fixed"
-                }
+                "matcher": {
+                  "id": "byName",
+                  "options": "Unknown"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               }
             ]
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "ReadOnly"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 8,
+            "y": 7
+          },
+          "id": 93,
+          "options": {
+            "legend": {
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "values": [
+                "value"
+              ]
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "orange",
-                  "mode": "fixed"
-                }
-              }
-            ]
+            "pieType": "pie",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Removing"
+          "pluginVersion": "9.3.6",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "region_num{type=\"DataRegion\", instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{status}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "DataRegion Current Status",
+          "type": "piechart"
+        },
+        {
+          "description": "The current status of cluster SchemaRegions",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                }
+              },
+              "mappings": []
             },
-            "properties": [
+            "overrides": [
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "red",
-                  "mode": "fixed"
-                }
+                "matcher": {
+                  "id": "byName",
+                  "options": "Running"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "ReadOnly"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Removing"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Unknown"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
               }
             ]
           },
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Unknown"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 12,
+            "y": 7
+          },
+          "id": 87,
+          "options": {
+            "legend": {
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "values": [
+                "value"
+              ]
             },
-            "properties": [
-              {
-                "id": "color",
-                "value": {
-                  "fixedColor": "yellow",
-                  "mode": "fixed"
-                }
-              }
-            ]
-          }
-        ]
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 12,
-        "y": 7
-      },
-      "id": 87,
-      "options": {
-        "legend": {
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "values": [
-            "value"
-          ]
-        },
-        "pieType": "pie",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+            "pieType": "pie",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "region_num{type=\"SchemaRegion\", instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{status}}",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "SchemaRegion Current Status",
+          "type": "piechart"
         },
-        "tooltip": {
-          "mode": "single",
-          "sort": "none"
-        }
-      },
-      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "builder",
-          "exemplar": true,
-          "expr": "region_num{type=\"SchemaRegion\", instance=\"$instance\"}",
-          "interval": "",
-          "legendFormat": "{{status}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "SchemaRegion Current Status",
-      "type": "piechart"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "description": "The system memory of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "description": "The system memory of current ConfigNode",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              },
+              "unit": "bytes"
+            },
+            "overrides": []
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 16,
+            "y": 7
           },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 16,
-        "y": 7
-      },
-      "id": 71,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "id": 71,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
+          },
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "System Memory",
+          "type": "stat"
         },
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "code",
-          "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
-          "legendFormat": "{{name}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "System Memory",
-      "type": "stat"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "description": "The swap memory of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "description": "The swap memory of current ConfigNode",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              },
+              "unit": "bytes"
+            },
+            "overrides": []
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 20,
+            "y": 7
           },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 4,
-        "x": 20,
-        "y": 7
-      },
-      "id": 75,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
-        },
-        "text": {},
-        "textMode": "auto"
-      },
-      "pluginVersion": "8.4.2",
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "id": 75,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "text": {},
+            "textMode": "auto"
           },
-          "editorMode": "builder",
-          "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
-          "legendFormat": "{{name}}",
-          "range": true,
-          "refId": "A"
+          "pluginVersion": "8.4.2",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "Swap Memory",
+          "type": "stat"
         }
       ],
-      "title": "Swap Memory",
-      "type": "stat"
+      "title": "Overview",
+      "type": "row"
     },
     {
       "collapsed": true,
@@ -1208,7 +1209,7 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 13
+        "y": 1
       },
       "id": 49,
       "panels": [
@@ -1252,8 +1253,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1339,8 +1339,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1487,8 +1486,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1635,8 +1633,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1722,8 +1719,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1809,8 +1805,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1896,8 +1891,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1953,7 +1947,7 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 14
+        "y": 2
       },
       "id": 53,
       "panels": [
@@ -1997,8 +1991,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -2009,7 +2002,7 @@
             "h": 6,
             "w": 8,
             "x": 0,
-            "y": 33
+            "y": 3
           },
           "id": 105,
           "options": {
@@ -2084,8 +2077,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -2096,7 +2088,7 @@
             "h": 6,
             "w": 8,
             "x": 8,
-            "y": 33
+            "y": 3
           },
           "id": 107,
           "options": {
@@ -2166,26 +2158,186 @@
                   "mode": "off"
                 }
               },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green"
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 6,
+            "w": 8,
+            "x": 16,
+            "y": 3
+          },
+          "id": 103,
+          "options": {
+            "legend": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "series_slot_num_in_database{instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "SeriesSlot Count",
+          "type": "timeseries"
+        }
+      ],
+      "title": "Database",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 3
+      },
+      "id": 81,
+      "panels": [
+        {
+          "description": "The status history of cluster DataRegions",
+          "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"
+                  }
+                ]
+              }
+            },
+            "overrides": [
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "ReadOnly"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Removing"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "red",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Running"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Unknown"
+                },
+                "properties": [
                   {
-                    "color": "green",
-                    "value": null
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
                   }
                 ]
               }
-            },
-            "overrides": []
+            ]
           },
           "gridPos": {
             "h": 6,
-            "w": 8,
-            "x": 16,
-            "y": 33
+            "w": 12,
+            "x": 0,
+            "y": 4
           },
-          "id": 103,
+          "id": 91,
           "options": {
             "legend": {
               "calcs": [
@@ -2208,32 +2360,18 @@
               },
               "editorMode": "builder",
               "exemplar": true,
-              "expr": "series_slot_num_in_database{instance=\"$instance\"}",
+              "expr": "region_num{type=\"DataRegion\", instance=\"$instance\"}",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{status}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "SeriesSlot Count",
+          "title": "DataRegion Status History",
           "type": "timeseries"
-        }
-      ],
-      "title": "Database",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 15
-      },
-      "id": 81,
-      "panels": [
+        },
         {
-          "description": "The status history of cluster DataRegions",
+          "description": "The status history of cluster SchemaRegions",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -2272,8 +2410,11 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
                   }
                 ]
               }
@@ -2310,231 +2451,770 @@
                 ]
               },
               {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Running"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "green",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
+                "matcher": {
+                  "id": "byName",
+                  "options": "Running"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "green",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
+              {
+                "matcher": {
+                  "id": "byName",
+                  "options": "Unknown"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              }
+            ]
+          },
+          "gridPos": {
+            "h": 6,
+            "w": 12,
+            "x": 12,
+            "y": 4
+          },
+          "id": 89,
+          "options": {
+            "legend": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "builder",
+              "exemplar": true,
+              "expr": "region_num{type=\"SchemaRegion\", instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{status}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "SchemaRegion Status History",
+          "type": "timeseries"
+        }
+      ],
+      "title": "Region",
+      "type": "row"
+    },
+    {
+      "collapsed": false,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "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"
+            },
+            "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": 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
               },
               {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Unknown"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "yellow",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
+                "color": "red",
+                "value": 80
               }
             ]
           },
-          "gridPos": {
-            "h": 6,
-            "w": 12,
-            "x": 0,
-            "y": 16
+          "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}"
           },
-          "id": 91,
-          "options": {
-            "legend": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": 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"
+        },
+        {
+          "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
             },
-            "tooltip": {
-              "mode": "single",
-              "sort": "none"
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
             }
           },
-          "targets": [
+          "mappings": [
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
+                }
               },
-              "editorMode": "builder",
-              "exemplar": true,
-              "expr": "region_num{type=\"DataRegion\", instance=\"$instance\"}",
-              "interval": "",
-              "legendFormat": "{{status}}",
-              "range": true,
-              "refId": "A"
+              "type": "special"
+            },
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
+              },
+              "type": "special"
             }
           ],
-          "title": "DataRegion Status History",
-          "type": "timeseries"
-        },
-        {
-          "description": "The status history of cluster SchemaRegions",
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
               },
-              "custom": {
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 10,
-                "gradientMode": "none",
-                "hideFrom": {
+              {
+                "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": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
+                  "viz": true
                 }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green",
-                    "value": null
-                  },
-                  {
-                    "color": "red",
-                    "value": 80
-                  }
-                ]
               }
+            ]
+          }
+        ]
+      },
+      "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
             },
-            "overrides": [
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "ReadOnly"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "orange",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
+            "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"
+                }
               },
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Removing"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "red",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
+              "type": "special"
+            },
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
               },
+              "type": "special"
+            }
+          ],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
               {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Running"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "green",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
+                "color": "green",
+                "value": null
               },
               {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Unknown"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "yellow",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
+                "color": "red",
+                "value": 80
               }
             ]
           },
-          "gridPos": {
-            "h": 6,
-            "w": 12,
-            "x": 12,
-            "y": 16
+          "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}"
           },
-          "id": 89,
-          "options": {
-            "legend": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
+          "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
             },
-            "tooltip": {
-              "mode": "single",
-              "sort": "none"
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
             }
           },
-          "targets": [
+          "mappings": [
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
+                }
               },
-              "editorMode": "builder",
-              "exemplar": true,
-              "expr": "region_num{type=\"SchemaRegion\", instance=\"$instance\"}",
-              "interval": "",
-              "legendFormat": "{{status}}",
-              "range": true,
-              "refId": "A"
+              "type": "special"
+            },
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
+              },
+              "type": "special"
             }
           ],
-          "title": "SchemaRegion Status History",
-          "type": "timeseries"
+          "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": "Region",
-      "type": "row"
+      "title": "RatisConsensus Memory",
+      "type": "timeseries"
     },
     {
       "collapsed": true,
@@ -2546,7 +3226,7 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 16
+        "y": 23
       },
       "id": 13,
       "panels": [
@@ -2595,8 +3275,7 @@
                 "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -2608,7 +3287,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 17
+            "y": 5
           },
           "id": 15,
           "options": {
@@ -2698,8 +3377,7 @@
                 "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -2711,7 +3389,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 17
+            "y": 5
           },
           "id": 16,
           "options": {
@@ -2786,8 +3464,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -2845,7 +3522,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 17
+            "y": 5
           },
           "id": 25,
           "options": {
@@ -2922,8 +3599,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -2935,7 +3611,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 25
+            "y": 13
           },
           "id": 17,
           "options": {
@@ -3037,8 +3713,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3081,7 +3756,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 25
+            "y": 13
           },
           "id": 18,
           "options": {
@@ -3170,8 +3845,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   },
                   {
                     "color": "red",
@@ -3187,7 +3861,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 25
+            "y": 13
           },
           "id": 19,
           "options": {
@@ -3285,8 +3959,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -3297,7 +3970,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 33
+            "y": 21
           },
           "id": 20,
           "options": {
@@ -3385,8 +4058,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3398,7 +4070,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 33
+            "y": 21
           },
           "id": 21,
           "options": {
@@ -3484,8 +4156,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -3496,7 +4167,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 33
+            "y": 21
           },
           "id": 22,
           "options": {
@@ -3584,8 +4255,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3597,7 +4267,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 41
+            "y": 29
           },
           "id": 23,
           "options": {
@@ -3721,8 +4391,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3734,7 +4403,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 41
+            "y": 29
           },
           "id": 24,
           "options": {
@@ -3813,8 +4482,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -3825,7 +4493,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 41
+            "y": 29
           },
           "id": 27,
           "options": {
@@ -3916,8 +4584,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3929,7 +4596,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 49
+            "y": 37
           },
           "id": 26,
           "options": {
@@ -3985,7 +4652,7 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 17
+        "y": 24
       },
       "id": 29,
       "panels": [
@@ -4034,8 +4701,7 @@
                 "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   },
                   {
                     "color": "#EAB839",
@@ -4055,7 +4721,7 @@
             "h": 8,
             "w": 12,
             "x": 0,
-            "y": 18
+            "y": 6
           },
           "id": 31,
           "options": {
@@ -4135,8 +4801,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4148,7 +4813,7 @@
             "h": 8,
             "w": 12,
             "x": 12,
-            "y": 18
+            "y": 6
           },
           "id": 33,
           "options": {
@@ -4228,8 +4893,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4241,7 +4905,7 @@
             "h": 8,
             "w": 12,
             "x": 0,
-            "y": 26
+            "y": 14
           },
           "id": 35,
           "options": {
@@ -4320,8 +4984,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4333,7 +4996,7 @@
             "h": 8,
             "w": 12,
             "x": 12,
-            "y": 26
+            "y": 14
           },
           "id": 37,
           "options": {
@@ -4408,8 +5071,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4421,7 +5083,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 34
+            "y": 22
           },
           "id": 39,
           "options": {
@@ -4502,8 +5164,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4515,7 +5176,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 34
+            "y": 22
           },
           "id": 41,
           "options": {
@@ -4593,8 +5254,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -4605,7 +5265,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 34
+            "y": 22
           },
           "id": 43,
           "options": {
@@ -4685,8 +5345,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4698,7 +5357,7 @@
             "h": 8,
             "w": 12,
             "x": 0,
-            "y": 42
+            "y": 30
           },
           "id": 47,
           "options": {
@@ -4774,8 +5433,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4787,7 +5445,7 @@
             "h": 8,
             "w": 12,
             "x": 12,
-            "y": 42
+            "y": 30
           },
           "id": 45,
           "options": {
@@ -4893,7 +5551,7 @@
     ]
   },
   "time": {
-    "from": "now-2d",
+    "from": "now-1h",
     "to": "now"
   },
   "timepicker": {
@@ -4909,6 +5567,6 @@
   "timezone": "browser",
   "title": "Apache IoTDB ConfigNode Dashboard",
   "uid": "4WkTYkx4z",
-  "version": 3,
+  "version": 15,
   "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 be728634bd..13faae9d4a 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
@@ -9,13 +9,13 @@
       "pluginName": "Prometheus"
     }
   ],
-  "__elements": {},
+  "__elements": [],
   "__requires": [
     {
       "type": "grafana",
       "id": "grafana",
       "name": "Grafana",
-      "version": "9.3.6"
+      "version": "8.4.2"
     },
     {
       "type": "datasource",
@@ -63,6 +63,7 @@
   "fiscalYearStartMonth": 0,
   "graphTooltip": 1,
   "id": null,
+  "iteration": 1678983382526,
   "links": [],
   "liveNow": false,
   "panels": [
@@ -70,7 +71,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -91,8 +92,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -136,7 +135,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 241
+            "y": 1
           },
           "id": 2,
           "options": {
@@ -180,8 +179,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -225,7 +222,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 241
+            "y": 1
           },
           "id": 3,
           "options": {
@@ -269,8 +266,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -315,7 +310,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 241
+            "y": 1
           },
           "id": 4,
           "options": {
@@ -353,7 +348,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -365,7 +360,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -386,8 +381,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -432,7 +425,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 11
+            "y": 10
           },
           "id": 6,
           "options": {
@@ -475,8 +468,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -521,7 +512,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 11
+            "y": 10
           },
           "id": 55,
           "options": {
@@ -564,8 +555,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -610,7 +599,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 11
+            "y": 10
           },
           "id": 56,
           "options": {
@@ -653,8 +642,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -699,7 +686,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 19
+            "y": 18
           },
           "id": 9,
           "options": {
@@ -743,8 +730,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -789,7 +774,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 19
+            "y": 18
           },
           "id": 10,
           "options": {
@@ -833,8 +818,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -881,7 +864,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 19
+            "y": 18
           },
           "id": 11,
           "options": {
@@ -981,7 +964,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 27
+            "y": 26
           },
           "id": 12,
           "options": {
@@ -1069,7 +1052,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 27
+            "y": 26
           },
           "id": 13,
           "options": {
@@ -1107,7 +1090,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -1119,7 +1102,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -1833,7 +1816,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -1845,7 +1828,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -5302,7 +5285,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -5314,7 +5297,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -9190,7 +9173,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -9202,7 +9185,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -10977,7 +10960,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -10989,7 +10972,7 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
@@ -11010,8 +10993,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11056,7 +11037,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 207
+            "y": 7
           },
           "id": 21,
           "options": {
@@ -11129,8 +11110,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11174,7 +11153,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 207
+            "y": 7
           },
           "id": 22,
           "options": {
@@ -11219,8 +11198,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11265,7 +11242,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 207
+            "y": 7
           },
           "id": 23,
           "options": {
@@ -11325,8 +11302,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11371,7 +11346,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 215
+            "y": 15
           },
           "id": 24,
           "options": {
@@ -11416,8 +11391,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11492,7 +11465,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 215
+            "y": 15
           },
           "id": 25,
           "options": {
@@ -11537,8 +11510,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11608,7 +11579,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 215
+            "y": 15
           },
           "id": 26,
           "options": {
@@ -11653,8 +11624,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11698,7 +11667,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 223
+            "y": 23
           },
           "id": 27,
           "options": {
@@ -11743,8 +11712,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11788,7 +11755,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 223
+            "y": 23
           },
           "id": 175,
           "options": {
@@ -11832,8 +11799,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11878,7 +11843,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 223
+            "y": 23
           },
           "id": 29,
           "options": {
@@ -11923,8 +11888,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -11969,7 +11932,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 231
+            "y": 31
           },
           "id": 30,
           "options": {
@@ -12016,8 +11979,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -12062,7 +12023,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 231
+            "y": 31
           },
           "id": 31,
           "options": {
@@ -12110,8 +12071,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -12156,7 +12115,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 231
+            "y": 31
           },
           "id": 32,
           "options": {
@@ -12204,8 +12163,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -12249,7 +12206,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 239
+            "y": 39
           },
           "id": 28,
           "options": {
@@ -12294,8 +12251,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -12339,7 +12294,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 239
+            "y": 39
           },
           "id": 173,
           "options": {
@@ -12377,7 +12332,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -12386,518 +12341,675 @@
       "type": "row"
     },
     {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
+      "collapsed": false,
       "gridPos": {
         "h": 1,
         "w": 24,
         "x": 0,
         "y": 7
       },
-      "id": 33,
-      "panels": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+      "id": 283,
+      "panels": [],
+      "title": "DataRegion Ratis Consensus",
+      "type": "row"
+    },
+    {
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
           },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "thresholds"
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "Core"
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 10,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
             },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 17
-          },
-          "id": 34,
-          "options": {
-            "colorMode": "value",
-            "graphMode": "area",
-            "justifyMode": "auto",
-            "orientation": "auto",
-            "reduceOptions": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "fields": "",
-              "values": false
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
             },
-            "textMode": "auto"
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
           },
-          "pluginVersion": "9.3.6",
-          "targets": [
+          "mappings": [
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sys_cpu_cores{instance=\"$instance\",name=\"system\"}",
-              "interval": "",
-              "legendFormat": "__auto",
-              "range": true,
-              "refId": "A"
+              "type": "special"
+            },
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
+              },
+              "type": "special"
             }
           ],
-          "title": "CPU Core",
-          "type": "stat"
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "s"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 12,
+        "x": 0,
+        "y": 8
+      },
+      "id": 288,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right"
         },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "desc"
+        }
+      },
+      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "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": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "percent"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 17
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_log_worker_appendEntryLatency_seconds_sum{instance=\"$instance\"}[1m])/rate(DataRegion_ratis_log_worker_appendEntryLatency_seconds_count{instance=\"$instance\"}[1m])",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "appendEntryLatency",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
           },
-          "id": 35,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_server_follower_append_entry_latency_seconds_sum{instance=\"$instance\"}[1m])/rate(DataRegion_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(DataRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeStateMachine\"}[1m]) / rate(DataRegion_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(DataRegion_ratis_server_clientWriteRequest_seconds_sum{instance=\"$instance\"}[1m]) / rate(DataRegion_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
             },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
             }
           },
-          "targets": [
+          "mappings": [
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
-              "interval": "",
-              "legendFormat": "System CPU Load",
-              "range": true,
-              "refId": "A"
+              "type": "special"
             },
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
-              "hide": false,
-              "legendFormat": "Process CPU Load",
-              "range": true,
-              "refId": "B"
+              "type": "special"
             }
           ],
-          "title": "CPU Load",
-          "type": "timeseries"
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "s"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 12,
+        "x": 12,
+        "y": 8
+      },
+      "id": 285,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right"
         },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "desc"
+        }
+      },
+      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "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": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "ns"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 17
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_log_worker_queueingDelay_seconds_sum{instance=\"$instance\"}[1m]) / rate(DataRegion_ratis_log_worker_queueingDelay_seconds_count{instance=\"$instance\"}[1m])",
+          "interval": "",
+          "legendFormat": "queueingDelay",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
           },
-          "id": 40,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "none"
-            }
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_log_worker_enqueuedTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(DataRegion_ratis_log_worker_enqueuedTime_seconds_count{instance=\"$instance\"}[1m])",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "enqueuedTime",
+          "refId": "B"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
           },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
-              "interval": "",
-              "legendFormat": "System CPU Load",
-              "range": true,
-              "refId": "A"
-            }
-          ],
-          "title": "CPU Time(per minute)",
-          "type": "timeseries"
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_log_worker_writelogExecutionTime_seconds_sum {instance=\"$instance\"}[1m]) / rate(DataRegion_ratis_log_worker_writelogExecutionTime_seconds_count{instance=\"$instance\"}[1m])",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "writelogExecutionTime",
+          "refId": "C"
         },
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "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": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "bytes"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 25
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_log_worker_flushTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(DataRegion_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"
           },
-          "id": 41,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 10,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
             },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
             }
           },
-          "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"
-            },
+          "mappings": [
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
-              "hide": false,
-              "legendFormat": "Total physical memory",
-              "range": true,
-              "refId": "B"
+              "type": "special"
             },
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "expr": "sys_committed_vm_size{instance=\"$instance\"}",
-              "hide": false,
-              "legendFormat": "Committed vm size",
-              "range": true,
-              "refId": "C"
+              "type": "special"
             }
           ],
-          "title": "System Memory",
-          "type": "timeseries"
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "s"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 8,
+        "x": 0,
+        "y": 17
+      },
+      "id": 291,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right"
         },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "desc"
+        }
+      },
+      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "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": "off"
-                }
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "percentage",
-                "steps": [
-                  {
-                    "color": "green"
-                  }
-                ]
-              },
-              "unit": "bytes"
-            },
-            "overrides": []
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeLocally\"}[1m]) / rate(DataRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "{{stage}}",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
           },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 25
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeRemotely\"}[1m]) / rate(DataRegion_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"
           },
-          "id": 42,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 10,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
             },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
             }
           },
-          "targets": [
+          "mappings": [
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"} - sys_free_swap_space_size{instance=\"$instance\", name=\"system\"}",
-              "interval": "",
-              "legendFormat": "Used Swap Size",
-              "range": true,
-              "refId": "A"
+              "type": "special"
             },
             {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
               },
-              "editorMode": "code",
-              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
-              "hide": false,
-              "legendFormat": "Total Swap Size",
-              "range": true,
-              "refId": "B"
+              "type": "special"
             }
           ],
-          "title": "System Swap Size",
-          "type": "timeseries"
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "reqps"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 8,
+        "x": 8,
+        "y": 17
+      },
+      "id": 292,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right"
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "desc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "exemplar": true,
+          "expr": "rate(DataRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "{{stage}}",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "exemplar": true,
+          "expr": "rate(DataRegion_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": 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
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "bytes"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 8,
+        "x": 16,
+        "y": 17
+      },
+      "id": 290,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "list",
+          "placement": "right"
+        },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "desc"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "exemplar": true,
+          "expr": "DataRegion_ratis_log_worker_closedSegmentsSizeInBytes {instance=\"$instance\"}",
+          "interval": "",
+          "legendFormat": "closedSegmentsSizeInBytes",
+          "refId": "A"
         },
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "exemplar": true,
+          "expr": "DataRegion_ratis_log_worker_openSegmentSizeInBytes  {instance=\"$instance\"}",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "openSegmentSizeInBytes",
+          "refId": "C"
+        }
+      ],
+      "title": "RatisConsensus Memory",
+      "type": "timeseries"
+    },
+    {
+      "collapsed": true,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 26
+      },
+      "id": 294,
+      "panels": [
+        {
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 0,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -12910,7 +13022,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -12920,32 +13032,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": "percentage",
+                "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green"
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
                   }
                 ]
               },
-              "unit": "bytes"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 25
+            "h": 9,
+            "w": 12,
+            "x": 0,
+            "y": 9
           },
-          "id": 43,
+          "id": 295,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right"
             },
             "tooltip": {
               "mode": "multi",
@@ -12958,12 +13097,11 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "process_max_mem{instance=~\"${instance}\"}",
+              "expr": "rate(SchemaRegion_ratis_log_worker_appendEntryLatency_seconds_sum{instance=\"$instance\"}[1m])/rate(SchemaRegion_ratis_log_worker_appendEntryLatency_seconds_count{instance=\"$instance\"}[1m])",
+              "hide": false,
               "interval": "",
-              "legendFormat": "Max Memory",
-              "range": true,
+              "legendFormat": "appendEntryLatency",
               "refId": "A"
             },
             {
@@ -12971,11 +13109,11 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "process_total_mem{instance=~\"${instance}\"}",
+              "exemplar": true,
+              "expr": "rate(SchemaRegion_ratis_server_follower_append_entry_latency_seconds_sum{instance=\"$instance\"}[1m])/rate(SchemaRegion_ratis_server_follower_append_entry_latency_seconds_count{instance=\"$instance\"}[1m])",
               "hide": false,
-              "legendFormat": "Total Memory",
-              "range": true,
+              "interval": "",
+              "legendFormat": "followerAppendEntryLatency",
               "refId": "B"
             },
             {
@@ -12983,35 +13121,41 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "process_total_mem{instance=~\"${instance}\"} - process_free_mem{instance=~\"${instance}\"}",
+              "exemplar": true,
+              "expr": "rate(SchemaRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeStateMachine\"}[1m]) / rate(SchemaRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeStateMachine\"}[1m])",
               "hide": false,
-              "legendFormat": "Used Memory",
-              "range": true,
+              "interval": "",
+              "legendFormat": "writeStateMachine",
               "refId": "C"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "rate(SchemaRegion_ratis_server_clientWriteRequest_seconds_sum{instance=\"$instance\"}[1m])/rate(SchemaRegion_ratis_server_clientWriteRequest_seconds_count{instance=\"$instance\"}[1m])",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "clientWriteRequest",
+              "refId": "D"
             }
           ],
-          "title": "Process Memory",
+          "title": "Ratis Stage 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,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -13024,7 +13168,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -13034,32 +13178,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"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 33
+            "h": 9,
+            "w": 12,
+            "x": 12,
+            "y": 9
           },
-          "id": 44,
+          "id": 296,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right"
             },
             "tooltip": {
               "mode": "multi",
@@ -13073,9 +13244,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "file_size{instance=~\"$instance\"}",
+              "expr": "rate(SchemaRegion_ratis_log_worker_queueingDelay_seconds_sum{instance=\"$instance\"}[1m]) / rate(SchemaRegion_ratis_log_worker_queueingDelay_seconds_count{instance=\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "queueingDelay",
               "refId": "A"
             },
             {
@@ -13084,34 +13255,52 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(file_size{instance=~\"$instance\"})",
+              "expr": "rate(SchemaRegion_ratis_log_worker_enqueuedTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(SchemaRegion_ratis_log_worker_enqueuedTime_seconds_count{instance=\"$instance\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "total size of file",
+              "legendFormat": "enqueuedTime",
               "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "rate(SchemaRegion_ratis_log_worker_writelogExecutionTime_seconds_sum {instance=\"$instance\"}[1m]) / rate(SchemaRegion_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(SchemaRegion_ratis_log_worker_flushTime_seconds_sum{instance=\"$instance\"}[1m]) / rate(SchemaRegion_ratis_log_worker_flushTime_seconds_count{instance=\"$instance\"}[1m])",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "flushTime",
+              "refId": "D"
             }
           ],
-          "title": "The Size Of File",
+          "title": "Write Log Entry",
           "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,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -13124,7 +13313,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -13134,56 +13323,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"
-                  }
-                ]
-              }
-            },
-            "overrides": [
-              {
-                "__systemRef": "hideSeriesFrom",
-                "matcher": {
-                  "id": "byNames",
-                  "options": {
-                    "mode": "exclude",
-                    "names": [
-                      "inner-seq-temp-num"
-                    ],
-                    "prefix": "All except:",
-                    "readOnly": true
-                  }
-                },
-                "properties": [
+                    "color": "green",
+                    "value": null
+                  },
                   {
-                    "id": "custom.hideFrom",
-                    "value": {
-                      "legend": false,
-                      "tooltip": false,
-                      "viz": true
-                    }
+                    "color": "red",
+                    "value": 80
                   }
                 ]
-              }
-            ]
+              },
+              "unit": "s"
+            },
+            "overrides": []
           },
           "gridPos": {
-            "h": 8,
+            "h": 9,
             "w": 8,
-            "x": 8,
-            "y": 33
+            "x": 0,
+            "y": 18
           },
-          "id": 45,
+          "id": 297,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right"
             },
             "tooltip": {
               "mode": "multi",
@@ -13197,9 +13389,10 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "file_count{instance=~\"$instance\"}",
+              "expr": "rate(SchemaRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeLocally\"}[1m]) / rate(SchemaRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
+              "hide": false,
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{stage}}",
               "refId": "A"
             },
             {
@@ -13208,35 +13401,28 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(file_count{instance=~\"$instance\"})",
+              "expr": "rate(SchemaRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeRemotely\"}[1m]) / rate(SchemaRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "total number of file",
+              "legendFormat": "{{stage}}",
               "refId": "B"
             }
           ],
-          "title": "The Number Of File",
+          "title": "Remote / Local Write Time",
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
-                "axisSoftMin": -5,
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 0,
+                "fillOpacity": 10,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -13249,7 +13435,7 @@
                 "scaleDistribution": {
                   "type": "linear"
                 },
-                "showPoints": "auto",
+                "showPoints": "never",
                 "spanNulls": false,
                 "stacking": {
                   "group": "A",
@@ -13259,32 +13445,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"
+              "unit": "reqps"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 8,
+            "h": 9,
             "w": 8,
-            "x": 16,
-            "y": 33
+            "x": 8,
+            "y": 18
           },
-          "id": 46,
+          "id": 300,
           "options": {
             "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right"
             },
             "tooltip": {
               "mode": "multi",
@@ -13297,12 +13510,11 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sys_disk_total_space{instance=~\"$instance\", name=\"system\"}-sys_disk_free_space{instance=\"$instance\", name=\"system\"}",
+              "expr": "rate(SchemaRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
+              "hide": false,
               "interval": "",
-              "legendFormat": "Used disk space",
-              "range": true,
+              "legendFormat": "{{stage}}",
               "refId": "A"
             },
             {
@@ -13310,32 +13522,24 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sys_disk_total_space{instance=~\"$instance\", name=\"system\"}",
+              "expr": "rate(SchemaRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
               "hide": false,
               "interval": "",
-              "legendFormat": "Total disk space",
-              "range": true,
+              "legendFormat": "{{stage}}",
               "refId": "B"
             }
           ],
-          "title": "The Space of Disk",
+          "title": "Remote / Local Write QPS",
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -13368,30 +13572,35 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green"
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
                   }
                 ]
-              }
+              },
+              "unit": "bytes"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 7,
+            "h": 9,
             "w": 8,
-            "x": 0,
-            "y": 41
+            "x": 16,
+            "y": 18
           },
-          "id": 47,
+          "id": 299,
           "options": {
             "legend": {
               "calcs": [],
               "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+              "placement": "right"
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "none"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -13400,12 +13609,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": "SchemaRegion_ratis_log_worker_closedSegmentsSizeInBytes {instance=\"$instance\"}",
               "interval": "",
-              "legendFormat": "Young GC number",
-              "range": true,
+              "legendFormat": "closedSegmentsSizeInBytes",
               "refId": "A"
             },
             {
@@ -13414,16 +13621,34 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+              "expr": "SchemaRegion_ratis_log_worker_openSegmentSizeInBytes{instance=\"$instance\"}",
               "hide": false,
               "interval": "",
-              "legendFormat": "Full GC number",
+              "legendFormat": "openSegmentSizeInBytes",
               "refId": "B"
             }
           ],
-          "title": "The Number of GC Per Minute",
+          "title": "RatisConsensus Memory",
           "type": "timeseries"
-        },
+        }
+      ],
+      "title": "SchemaRegion Ratis Consensus",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 27
+      },
+      "id": 33,
+      "panels": [
         {
           "datasource": {
             "type": "prometheus",
@@ -13432,58 +13657,123 @@
           "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"
-                }
+                "mode": "thresholds"
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "Core"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 7,
+            "h": 8,
+            "w": 8,
+            "x": 0,
+            "y": 17
+          },
+          "id": 34,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "area",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
+          },
+          "pluginVersion": "9.3.6",
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sys_cpu_cores{instance=\"$instance\",name=\"system\"}",
+              "interval": "",
+              "legendFormat": "__auto",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "CPU Core",
+          "type": "stat"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "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": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "percentage",
+                "steps": [
+                  {
+                    "color": "green"
+                  }
+                ]
+              },
+              "unit": "percent"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
             "w": 8,
             "x": 8,
-            "y": 41
+            "y": 17
           },
-          "id": 48,
+          "id": 35,
           "options": {
             "legend": {
               "calcs": [],
@@ -13502,10 +13792,12 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
+              "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
               "interval": "",
-              "legendFormat": "The time of young GC",
+              "legendFormat": "System CPU Load",
+              "range": true,
               "refId": "A"
             },
             {
@@ -13513,15 +13805,15 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "exemplar": true,
-              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+              "editorMode": "code",
+              "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
               "hide": false,
-              "interval": "",
-              "legendFormat": "The time of full GC",
+              "legendFormat": "Process CPU Load",
+              "range": true,
               "refId": "B"
             }
           ],
-          "title": "The Time Consumed Of GC Per Minute",
+          "title": "CPU Load",
           "type": "timeseries"
         },
         {
@@ -13539,6 +13831,8 @@
                 "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 0,
@@ -13566,23 +13860,24 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "ns"
             },
             "overrides": []
           },
           "gridPos": {
-            "h": 7,
+            "h": 8,
             "w": 8,
             "x": 16,
-            "y": 41
+            "y": 17
           },
-          "id": 49,
+          "id": 40,
           "options": {
             "legend": {
               "calcs": [],
@@ -13601,26 +13896,16 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "jvm_threads_live_threads{instance=~\"$instance\"}",
+              "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "The total number of jvm thread",
+              "legendFormat": "System CPU Load",
+              "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": "The Number Of Java Thread",
+          "title": "CPU Time(per minute)",
           "type": "timeseries"
         },
         {
@@ -13638,6 +13923,8 @@
                 "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 0,
@@ -13665,7 +13952,7 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
@@ -13680,9 +13967,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 48
+            "y": 25
           },
-          "id": 50,
+          "id": 41,
           "options": {
             "legend": {
               "calcs": [],
@@ -13692,7 +13979,7 @@
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "none"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -13701,10 +13988,12 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(jvm_memory_max_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}-sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"}",
               "interval": "",
-              "legendFormat": "Maximum heap memory",
+              "legendFormat": "Used physical memory",
+              "range": true,
               "refId": "A"
             },
             {
@@ -13712,11 +14001,11 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "exemplar": true,
-              "expr": "sum(jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "editorMode": "code",
+              "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
               "hide": false,
-              "interval": "",
-              "legendFormat": "Used heap memory",
+              "legendFormat": "Total physical memory",
+              "range": true,
               "refId": "B"
             },
             {
@@ -13724,39 +14013,15 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "exemplar": true,
-              "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Old Gen\"}",
+              "editorMode": "code",
+              "expr": "sys_committed_vm_size{instance=\"$instance\"}",
               "hide": false,
-              "interval": "",
-              "legendFormat": "Old area",
+              "legendFormat": "Committed vm size",
+              "range": true,
               "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": "System Memory",
           "type": "timeseries"
         },
         {
@@ -13774,6 +14039,8 @@
                 "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 0,
@@ -13801,7 +14068,7 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
@@ -13816,9 +14083,9 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 48
+            "y": 25
           },
-          "id": 51,
+          "id": 42,
           "options": {
             "legend": {
               "calcs": [],
@@ -13837,14 +14104,28 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(jvm_buffer_memory_used_bytes{instance=~\"$instance\"})",
+              "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"} - sys_free_swap_space_size{instance=\"$instance\", name=\"system\"}",
               "interval": "",
-              "legendFormat": "off heap memory",
+              "legendFormat": "Used Swap Size",
+              "range": true,
               "refId": "A"
+            },
+            {
+              "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": "B"
             }
           ],
-          "title": "Off Heap Memory",
+          "title": "System Swap Size",
           "type": "timeseries"
         },
         {
@@ -13862,6 +14143,8 @@
                 "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 0,
@@ -13889,70 +14172,24 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
                     "color": "green"
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "bytes"
             },
-            "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": 48
+            "y": 25
           },
-          "id": 52,
+          "id": 43,
           "options": {
             "legend": {
               "calcs": [],
@@ -13973,14 +14210,38 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
+              "expr": "process_max_mem{instance=~\"${instance}\"}",
               "interval": "",
-              "legendFormat": "{{level}}",
+              "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": "Log Number Per Minute",
+          "title": "Process Memory",
           "type": "timeseries"
         },
         {
@@ -13988,7 +14249,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -14033,7 +14293,7 @@
                   }
                 ]
               },
-              "unit": "ms"
+              "unit": "bytes"
             },
             "overrides": []
           },
@@ -14041,9 +14301,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 56
+            "y": 33
           },
-          "id": 53,
+          "id": 44,
           "options": {
             "legend": {
               "calcs": [],
@@ -14062,16 +14322,150 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
+              "expr": "file_size{instance=~\"$instance\"}",
               "interval": "",
-              "legendFormat": "compiler",
-              "range": true,
+              "legendFormat": "{{name}}",
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(file_size{instance=~\"$instance\"})",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "total size of file",
+              "refId": "B"
             }
           ],
-          "title": "The Time Consumed Of Compilation Per Minute",
+          "title": "The Size Of File",
+          "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"
+                  }
+                ]
+              }
+            },
+            "overrides": [
+              {
+                "__systemRef": "hideSeriesFrom",
+                "matcher": {
+                  "id": "byNames",
+                  "options": {
+                    "mode": "exclude",
+                    "names": [
+                      "inner-seq-temp-num"
+                    ],
+                    "prefix": "All except:",
+                    "readOnly": true
+                  }
+                },
+                "properties": [
+                  {
+                    "id": "custom.hideFrom",
+                    "value": {
+                      "legend": false,
+                      "tooltip": false,
+                      "viz": true
+                    }
+                  }
+                ]
+              }
+            ]
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 8,
+            "x": 8,
+            "y": 33
+          },
+          "id": 45,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
+            },
+            "tooltip": {
+              "mode": "multi",
+              "sort": "desc"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "file_count{instance=~\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(file_count{instance=~\"$instance\"})",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "total number of file",
+              "refId": "B"
+            }
+          ],
+          "title": "The Number Of File",
           "type": "timeseries"
         },
         {
@@ -14079,7 +14473,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -14090,6 +14483,7 @@
                 "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMin": -5,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 0,
@@ -14124,17 +14518,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "bytes"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 8,
-            "x": 8,
-            "y": 56
+            "x": 16,
+            "y": 33
           },
-          "id": 54,
+          "id": 46,
           "options": {
             "legend": {
               "calcs": [],
@@ -14155,9 +14549,9 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
+              "expr": "sys_disk_total_space{instance=~\"$instance\", name=\"system\"}-sys_disk_free_space{instance=\"$instance\", name=\"system\"}",
               "interval": "",
-              "legendFormat": "Unloaded Number",
+              "legendFormat": "Used disk space",
               "range": true,
               "refId": "A"
             },
@@ -14167,806 +14561,1643 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "code",
-              "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+              "exemplar": true,
+              "expr": "sys_disk_total_space{instance=~\"$instance\", name=\"system\"}",
               "hide": false,
-              "legendFormat": "Loaded Number",
+              "interval": "",
+              "legendFormat": "Total disk space",
               "range": true,
               "refId": "B"
             }
           ],
-          "title": "The Number Of Class",
+          "title": "The Space of Disk",
           "type": "timeseries"
-        }
-      ],
-      "targets": [
+        },
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
-          "refId": "A"
-        }
-      ],
-      "title": "System",
-      "type": "row"
-    },
-    {
-      "collapsed": false,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 8
-      },
-      "id": 177,
-      "panels": [],
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+          "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"
+                  }
+                ]
+              }
+            },
+            "overrides": []
           },
-          "refId": "A"
-        }
-      ],
-      "title": "Disk Status",
-      "type": "row"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
+          "gridPos": {
+            "h": 7,
+            "w": 8,
+            "x": 0,
+            "y": 41
           },
-          "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"
+          "id": 47,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
-            "thresholdsStyle": {
-              "mode": "line+area"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "none"
             }
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "percentage",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
               },
-              {
-                "color": "#EAB839",
-                "value": 80
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
+              "interval": "",
+              "legendFormat": "Young GC number",
+              "range": true,
+              "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
               },
-              {
-                "color": "red",
-                "value": 90
-              }
-            ]
-          },
-          "unit": "percentunit"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 0,
-        "y": 9
-      },
-      "id": 179,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean",
-            "last"
+              "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"
+            }
           ],
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true
+          "title": "The Number of GC Per Minute",
+          "type": "timeseries"
         },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "asc"
-        }
-      },
-      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "code",
-          "exemplar": true,
-          "expr": "disk_io_busy_percentage{instance=~\"$instance\"}",
-          "interval": "",
-          "legendFormat": "{{name}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "Disk I/O Busy Rate",
-      "type": "timeseries"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
+          "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": "s"
+            },
+            "overrides": []
           },
-          "custom": {
-            "axisCenteredZero": false,
-            "axisColorMode": "text",
-            "axisLabel": "",
-            "axisPlacement": "auto",
-            "barAlignment": 0,
-            "drawStyle": "line",
-            "fillOpacity": 5,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
+          "gridPos": {
+            "h": 7,
+            "w": 8,
+            "x": 8,
+            "y": 41
+          },
+          "id": 48,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "desc"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
+              "interval": "",
+              "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": "The Time Consumed Of GC Per Minute",
+          "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"
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 7,
+            "w": 8,
+            "x": 16,
+            "y": 41
+          },
+          "id": 49,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
+            },
+            "tooltip": {
+              "mode": "multi",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "jvm_threads_live_threads{instance=~\"$instance\"}",
+              "interval": "",
+              "legendFormat": "The total number of jvm thread",
+              "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": "The Number Of Java Thread",
+          "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": "bytes"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 8,
+            "x": 0,
+            "y": 48
+          },
+          "id": 50,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
+            },
+            "tooltip": {
+              "mode": "multi",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(jvm_memory_max_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "interval": "",
+              "legendFormat": "Maximum heap memory",
+              "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\"})",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Used heap memory",
+              "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",
+          "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": "bytes"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 8,
+            "x": 8,
+            "y": 48
+          },
+          "id": 51,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
+            },
+            "tooltip": {
+              "mode": "multi",
+              "sort": "desc"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(jvm_buffer_memory_used_bytes{instance=~\"$instance\"})",
+              "interval": "",
+              "legendFormat": "off heap memory",
+              "refId": "A"
+            }
+          ],
+          "title": "Off Heap Memory",
+          "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": "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": 48
+          },
+          "id": 52,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
+              "showLegend": true
             },
-            "showPoints": "auto",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "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",
+              "interval": "",
+              "legendFormat": "{{level}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "Log Number Per Minute",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "description": "",
+          "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": "ms"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 8,
+            "x": 0,
+            "y": 56
+          },
+          "id": 53,
+          "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(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
+              "interval": "",
+              "legendFormat": "compiler",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "The Time Consumed Of Compilation Per Minute",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "description": "",
+          "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": "none"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 8,
+            "x": 8,
+            "y": 56
+          },
+          "id": 54,
+          "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": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
+              "interval": "",
+              "legendFormat": "Unloaded Number",
+              "range": true,
+              "refId": "A"
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+              "hide": false,
+              "legendFormat": "Loaded Number",
+              "range": true,
+              "refId": "B"
             }
-          },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
-          },
-          "unit": "KBs"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 12,
-        "y": 9
-      },
-      "id": 181,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean",
-            "last"
           ],
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true
-        },
-        "tooltip": {
-          "mode": "none",
-          "sort": "none"
+          "title": "The Number Of Class",
+          "type": "timeseries"
         }
-      },
+      ],
       "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"
+      "title": "System",
+      "type": "row"
     },
     {
+      "collapsed": true,
       "datasource": {
         "type": "prometheus",
         "uid": "${DS_PROMETHEUS}"
       },
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 28
+      },
+      "id": 177,
+      "panels": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
           },
-          "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"
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "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": [],
+              "thresholds": {
+                "mode": "percentage",
+                "steps": [
+                  {
+                    "color": "green"
+                  },
+                  {
+                    "color": "#EAB839",
+                    "value": 80
+                  },
+                  {
+                    "color": "red",
+                    "value": 90
+                  }
+                ]
+              },
+              "unit": "percentunit"
             },
-            "showPoints": "auto",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 0,
+            "y": 9
+          },
+          "id": 179,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean",
+                "last"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "asc"
             }
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
-          },
-          "unit": "iops"
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "disk_io_busy_percentage{instance=~\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "Disk I/O Busy Rate",
+          "type": "timeseries"
         },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 0,
-        "y": 17
-      },
-      "id": 183,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean",
-            "max"
+        {
+          "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": "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": 9
+          },
+          "id": 181,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean",
+                "last"
+              ],
+              "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"
+            }
           ],
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "sortBy": "Mean",
-          "sortDesc": true
+          "title": "Disk I/O Throughput",
+          "type": "timeseries"
         },
-        "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"
+          "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": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green"
+                  }
+                ]
+              },
+              "unit": "iops"
             },
-            "showPoints": "auto",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 0,
+            "y": 17
+          },
+          "id": 183,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean",
+                "max"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "sortBy": "Mean",
+              "sortDesc": true
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "asc"
             }
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
-          }
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 12,
-        "y": 17
-      },
-      "id": 193,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "rate(disk_io_ops{instance=~\"$instance\"}[1m])",
+              "interval": "",
+              "legendFormat": "{{name}}-{{type}}",
+              "refId": "A"
+            }
           ],
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "sortBy": "Mean",
-          "sortDesc": true
+          "title": "Disk I/O Ops",
+          "type": "timeseries"
         },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "asc"
-        }
-      },
-      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "code",
-          "exemplar": true,
-          "expr": "disk_io_avg_queue_size{instance=~\"$instance\"}",
-          "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"
+          "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": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green"
+                  }
+                ]
+              }
             },
-            "showPoints": "auto",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 12,
+            "y": 17
+          },
+          "id": 193,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "sortBy": "Mean",
+              "sortDesc": true
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "asc"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "disk_io_avg_queue_size{instance=~\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
             }
-          },
-          "decimals": 2,
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green"
-              }
-            ]
-          },
-          "unit": "ms"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 0,
-        "y": 25
-      },
-      "id": 187,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean",
-            "max"
           ],
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "sortBy": "Mean",
-          "sortDesc": true
+          "title": "Disk I/O Avg Queue Size",
+          "type": "timeseries"
         },
-        "tooltip": {
-          "mode": "multi",
-          "sort": "asc"
-        }
-      },
-      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "code",
-          "exemplar": true,
-          "expr": "disk_io_avg_time{instance=~\"$instance\"}",
-          "interval": "",
-          "legendFormat": "{{name}}-{{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": 5,
-            "gradientMode": "none",
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
-            },
-            "lineInterpolation": "linear",
-            "lineWidth": 1,
-            "pointSize": 5,
-            "scaleDistribution": {
-              "type": "linear"
+          "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": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "decimals": 2,
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green"
+                  }
+                ]
+              },
+              "unit": "ms"
             },
-            "showPoints": "auto",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 0,
+            "y": 25
+          },
+          "id": 187,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean",
+                "max"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "sortBy": "Mean",
+              "sortDesc": true
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "asc"
             }
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green"
-              }
-            ]
-          },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 12,
-        "y": 25
-      },
-      "id": 189,
-      "options": {
-        "legend": {
-          "calcs": [
-            "mean"
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "disk_io_avg_time{instance=~\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{name}}-{{type}}",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "sortBy": "Mean",
-          "sortDesc": true
+          "title": "Disk Per I/O Avg Time",
+          "type": "timeseries"
         },
-        "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"
+          "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": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green"
+                  }
+                ]
+              },
+              "unit": "bytes"
             },
-            "showPoints": "auto",
-            "spanNulls": false,
-            "stacking": {
-              "group": "A",
-              "mode": "none"
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 12,
+            "y": 25
+          },
+          "id": 189,
+          "options": {
+            "legend": {
+              "calcs": [
+                "mean"
+              ],
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "sortBy": "Mean",
+              "sortDesc": true
             },
-            "thresholdsStyle": {
-              "mode": "off"
+            "tooltip": {
+              "mode": "multi",
+              "sort": "asc"
             }
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green"
-              }
-            ]
-          },
-          "unit": "ops"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 8,
-        "w": 12,
-        "x": 0,
... 710 lines suppressed ...