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

[iotdb] 12/13: [To rel/1.1][IOTDB-5618] Add RatisConsensus metrics in dashboard (#9369)

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

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

commit f5ce7762e6c5b0c5dce357d70e7fbac09d68f81e
Author: Xiangpeng Hu <65...@users.noreply.github.com>
AuthorDate: Sun Mar 19 22:12:22 2023 +0800

    [To rel/1.1][IOTDB-5618] Add RatisConsensus metrics in dashboard (#9369)
---
 .../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         | 3837 +++++----
 .../Apache-IoTDB-DataNode-Dashboard.json           | 8781 +++++++++++---------
 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, 7463 insertions(+), 5652 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 a767d05c9c..16302fac47 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;
@@ -92,7 +91,6 @@ public class ConsensusManager {
                       .setThisNode(
                           new TEndPoint(CONF.getInternalAddress(), CONF.getConsensusPort()))
                       .setStorageDir(CONF.getConsensusDir())
-                      .setConsensusGroupType(TConsensusGroupType.ConfigRegion)
                       .build(),
                   gid -> stateMachine)
               .orElseThrow(
@@ -181,7 +179,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 89dfb35d65..7016071f59 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
@@ -9,7 +9,7 @@
       "pluginName": "Prometheus"
     }
   ],
-  "__elements": {},
+  "__elements": [],
   "__requires": [
     {
       "type": "panel",
@@ -21,7 +21,7 @@
       "type": "grafana",
       "id": "grafana",
       "name": "Grafana",
-      "version": "9.3.6"
+      "version": "8.4.2"
     },
     {
       "type": "panel",
@@ -75,11 +75,12 @@
   "fiscalYearStartMonth": 0,
   "graphTooltip": 1,
   "id": null,
+  "iteration": 1678983654736,
   "links": [],
   "liveNow": false,
   "panels": [
     {
-      "collapsed": false,
+      "collapsed": true,
       "gridPos": {
         "h": 1,
         "w": 24,
@@ -87,692 +88,1120 @@
         "y": 0
       },
       "id": 57,
-      "panels": [],
-      "title": "Overview",
-      "type": "row"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "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": "Unknown"
-            },
-            "properties": [
+                "matcher": {
+                  "id": "byName",
+                  "options": "ReadOnly"
+                },
+                "properties": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "orange",
+                      "mode": "fixed"
+                    }
+                  }
+                ]
+              },
               {
-                "id": "color",
-                "value": {
-                  "fixedColor": "yellow",
-                  "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"
+                    }
+                  }
+                ]
               }
             ]
-          }
-        ]
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 5,
-        "x": 0,
-        "y": 1
-      },
-      "id": 59,
-      "options": {
-        "legend": {
-          "displayMode": "table",
-          "placement": "right",
-          "showLegend": true,
-          "values": [
-            "value"
-          ]
-        },
-        "pieType": "pie",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
-        },
-        "tooltip": {
-          "mode": "single",
-          "sort": "none"
-        }
-      },
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "builder",
-          "expr": "node_num{type=\"ConfigNode\"}",
-          "legendFormat": "{{status}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "ConfigNode Current Status",
-      "type": "piechart"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "description": "The CPU Core of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 0,
+            "y": 1
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
+          "id": 59,
+          "options": {
+            "legend": {
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true,
+              "values": [
+                "value"
+              ]
+            },
+            "pieType": "pie",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
           },
-          "unit": "Core(s)"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 5,
-        "x": 5,
-        "y": 1
-      },
-      "id": 69,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "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"
         },
-        "textMode": "auto"
-      },
-      "pluginVersion": "9.3.6",
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "description": "The number of cluster Databases",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              }
+            },
+            "overrides": []
           },
-          "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"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "description": "The total disk space of current ConfigNode",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "thresholds"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 4,
+            "y": 1
           },
-          "mappings": [],
-          "thresholds": {
-            "mode": "absolute",
-            "steps": [
-              {
-                "color": "green",
-                "value": null
-              }
-            ]
+          "id": 79,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "area",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
           },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 5,
-        "x": 10,
-        "y": 1
-      },
-      "id": 77,
-      "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": "database_num{instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "__auto",
+              "range": true,
+              "refId": "A"
+            }
           ],
-          "fields": "",
-          "values": false
+          "title": "Database Count",
+          "type": "stat"
         },
-        "textMode": "auto"
-      },
-      "pluginVersion": "9.3.6",
-      "targets": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "editorMode": "builder",
-          "expr": "sys_disk_total_space",
-          "legendFormat": "{{name}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "Total Disk Space",
-      "type": "stat"
-    },
-    {
-      "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
+          "description": "The number of cluster DataRegions",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
               },
-              {
-                "color": "red",
-                "value": 90
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
               }
-            ]
+            },
+            "overrides": []
           },
-          "unit": "percentunit"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 12,
-        "w": 9,
-        "x": 15,
-        "y": 1
-      },
-      "id": 73,
-      "options": {
-        "displayMode": "lcd",
-        "minVizHeight": 10,
-        "minVizWidth": 0,
-        "orientation": "horizontal",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 8,
+            "y": 1
+          },
+          "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"
         },
-        "showUnfilled": true
-      },
-      "pluginVersion": "9.3.6",
-      "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": "code",
-          "expr": "process_cpu_load{instance=~\"$instance\"} / 100",
-          "hide": false,
-          "legendFormat": "cpu load",
-          "range": true,
-          "refId": "C"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 12,
+            "y": 1
           },
-          "editorMode": "code",
-          "expr": "1- sys_disk_free_space / sys_disk_total_space",
-          "hide": false,
-          "legendFormat": "disk",
-          "range": true,
-          "refId": "D"
-        },
+          "id": 83,
+          "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=\"SchemaRegion\", instance=\"$instance\"})",
+              "interval": "",
+              "legendFormat": "__auto",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "SchemaRegion Count",
+          "type": "stat"
+        },
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "editorMode": "code",
-          "expr": "1 - process_free_mem{instance=\"$instance\"} / process_total_mem{instance=\"$instance\"}",
-          "hide": false,
-          "legendFormat": "process memory",
-          "range": true,
-          "refId": "B"
+          "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": []
+          },
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 16,
+            "y": 1
+          },
+          "id": 69,
+          "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",
+              "exemplar": false,
+              "expr": "sys_cpu_cores{instance=\"$instance\", name=\"system\"}",
+              "instant": false,
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "CPU Core",
+          "type": "stat"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+          "description": "The total disk space of current ConfigNode",
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              },
+              "unit": "bytes"
+            },
+            "overrides": []
           },
-          "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"
-    },
-    {
-      "datasource": {
-        "type": "prometheus",
-        "uid": "${DS_PROMETHEUS}"
-      },
-      "description": "The current status of cluster DataNodes",
-      "fieldConfig": {
-        "defaults": {
-          "color": {
-            "mode": "palette-classic"
+          "gridPos": {
+            "h": 6,
+            "w": 4,
+            "x": 20,
+            "y": 1
           },
-          "custom": {
-            "hideFrom": {
-              "legend": false,
-              "tooltip": false,
-              "viz": false
+          "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}"
+              },
+              "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"
+              },
+              "custom": {
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                }
+              },
+              "mappings": []
+            },
+            "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"
+                    }
+                  }
+                ]
+              },
+              {
+                "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"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
             }
           },
-          "mappings": []
+          "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"
         },
-        "overrides": [
-          {
-            "matcher": {
-              "id": "byName",
-              "options": "Running"
+        {
+          "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"
             },
-            "properties": [
+            "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": [
+                  {
+                    "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": 5,
-        "x": 0,
-        "y": 7
-      },
-      "id": 65,
-      "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",
-          "expr": "node_num{type=\"DataNode\"}",
-          "legendFormat": "{{status}}",
-          "range": true,
-          "refId": "A"
-        }
-      ],
-      "title": "DataNode 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
+          },
+          "id": 71,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "none",
+            "justifyMode": "auto",
+            "orientation": "auto",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "textMode": "auto"
           },
-          "unit": "bytes"
-        },
-        "overrides": []
-      },
-      "gridPos": {
-        "h": 6,
-        "w": 5,
-        "x": 5,
-        "y": 7
-      },
-      "id": 71,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
+          "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": "9.3.6",
-      "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": 5,
-        "x": 10,
-        "y": 7
-      },
-      "id": 75,
-      "options": {
-        "colorMode": "value",
-        "graphMode": "none",
-        "justifyMode": "auto",
-        "orientation": "auto",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
-        },
-        "text": {},
-        "textMode": "auto"
-      },
-      "pluginVersion": "9.3.6",
-      "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,
@@ -780,28 +1209,51 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 13
+        "y": 1
       },
       "id": 49,
       "panels": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "description": "The number of cluster ConfigNodes",
           "fieldConfig": {
             "defaults": {
               "color": {
-                "mode": "thresholds"
+                "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",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -810,26 +1262,25 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 6,
+            "w": 7,
             "x": 0,
             "y": 14
           },
           "id": 55,
           "options": {
-            "colorMode": "value",
-            "graphMode": "area",
-            "justifyMode": "auto",
-            "orientation": "auto",
-            "reduceOptions": {
+            "legend": {
               "calcs": [
                 "lastNotNull"
               ],
-              "fields": "",
-              "values": false
+              "displayMode": "table",
+              "placement": "right"
             },
-            "textMode": "auto"
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
           },
-          "pluginVersion": "9.3.6",
+          "pluginVersion": "8.4.2",
           "targets": [
             {
               "datasource": {
@@ -837,20 +1288,18 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "sum by(type) (node_num{type=\"ConfigNode\"})",
-              "legendFormat": "__auto",
+              "exemplar": true,
+              "expr": "sum(node_num{instance=\"$instance\"}) by(type) ",
+              "interval": "",
+              "legendFormat": "{{type}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "ConfigNode Count",
-          "type": "stat"
+          "title": "Node Count",
+          "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "description": "The status history of cluster ConfigNodes",
           "fieldConfig": {
             "defaults": {
@@ -858,8 +1307,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -892,8 +1339,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -963,8 +1409,8 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 18,
-            "x": 6,
+            "w": 9,
+            "x": 7,
             "y": 14
           },
           "id": 61,
@@ -989,7 +1435,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "node_num{type=\"ConfigNode\"}",
+              "exemplar": true,
+              "expr": "node_num{type=\"ConfigNode\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{status}}",
               "range": true,
               "refId": "A"
@@ -997,88 +1445,8 @@
           ],
           "title": "ConfigNode Status History",
           "type": "timeseries"
-        }
-      ],
-      "title": "ConfigNode",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 14
-      },
-      "id": 51,
-      "panels": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "description": "The number of cluster DataNodes",
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "thresholds"
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green",
-                    "value": null
-                  }
-                ]
-              }
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 6,
-            "w": 6,
-            "x": 0,
-            "y": 15
-          },
-          "id": 63,
-          "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": "builder",
-              "expr": "sum by(type) (node_num{type=\"DataNode\"})",
-              "legendFormat": "__auto",
-              "range": true,
-              "refId": "A"
-            }
-          ],
-          "title": "DataNode Count",
-          "type": "stat"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
+        },
+        {
           "description": "The status history of cluster DataNodes",
           "fieldConfig": {
             "defaults": {
@@ -1086,8 +1454,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1120,8 +1486,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1191,9 +1556,9 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 18,
-            "x": 6,
-            "y": 15
+            "w": 8,
+            "x": 16,
+            "y": 14
           },
           "id": 67,
           "options": {
@@ -1217,7 +1582,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "node_num{type=\"DataNode\"}",
+              "exemplar": true,
+              "expr": "node_num{type=\"DataNode\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{status}}",
               "range": true,
               "refId": "A"
@@ -1227,10 +1594,6 @@
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "description": "The number of SchemaRegions in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -1238,8 +1601,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1272,8 +1633,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1284,7 +1644,7 @@
             "h": 6,
             "w": 12,
             "x": 0,
-            "y": 21
+            "y": 20
           },
           "id": 97,
           "options": {
@@ -1308,7 +1668,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_num_in_data_node{type=\"SchemaRegion\"}",
+              "exemplar": true,
+              "expr": "region_num_in_data_node{type=\"SchemaRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
@@ -1318,10 +1680,6 @@
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "description": "The number of SchemaRegionGroup leadere in each DataNodes",
           "fieldConfig": {
             "defaults": {
@@ -1329,8 +1687,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1363,8 +1719,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1375,7 +1730,7 @@
             "h": 6,
             "w": 12,
             "x": 12,
-            "y": 21
+            "y": 20
           },
           "id": 95,
           "options": {
@@ -1399,7 +1754,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_group_leader_num_in_data_node{type=\"SchemaRegion\"}",
+              "exemplar": true,
+              "expr": "region_group_leader_num_in_data_node{type=\"SchemaRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
@@ -1409,10 +1766,6 @@
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "description": "The number of DataRegions in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -1420,8 +1773,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1454,8 +1805,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1466,7 +1816,7 @@
             "h": 6,
             "w": 12,
             "x": 0,
-            "y": 27
+            "y": 26
           },
           "id": 99,
           "options": {
@@ -1490,7 +1840,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_num_in_data_node{type=\"DataRegion\"}",
+              "exemplar": true,
+              "expr": "region_num_in_data_node{type=\"DataRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
@@ -1500,10 +1852,6 @@
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
           "description": "The number of DataRegionGroup leaders in each DataNode",
           "fieldConfig": {
             "defaults": {
@@ -1511,8 +1859,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1545,8 +1891,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1557,7 +1902,7 @@
             "h": 6,
             "w": 12,
             "x": 12,
-            "y": 27
+            "y": 26
           },
           "id": 101,
           "options": {
@@ -1581,7 +1926,9 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_num_in_data_node{type=\"DataRegion\"}",
+              "exemplar": true,
+              "expr": "region_num_in_data_node{type=\"DataRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
@@ -1591,7 +1938,7 @@
           "type": "timeseries"
         }
       ],
-      "title": "DataNode",
+      "title": "NodeInfo",
       "type": "row"
     },
     {
@@ -1600,91 +1947,23 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 15
+        "y": 2
       },
       "id": 53,
       "panels": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "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": 6,
-            "x": 0,
-            "y": 16
-          },
-          "id": 79,
-          "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": "builder",
-              "expr": "database_num",
-              "legendFormat": "__auto",
-              "range": true,
-              "refId": "A"
-            }
-          ],
-          "title": "Database Count",
-          "type": "stat"
-        },
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "description": "The number of SeriesSlots in each Database",
+          "description": "The number of SchemaRegionGroups in each Database",
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 10,
+                "fillOpacity": 4,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -1712,8 +1991,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1722,11 +2000,11 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 18,
-            "x": 6,
-            "y": 16
+            "w": 8,
+            "x": 0,
+            "y": 3
           },
-          "id": 103,
+          "id": 105,
           "options": {
             "legend": {
               "calcs": [
@@ -1748,34 +2026,30 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "series_slot_num_in_database",
+              "exemplar": true,
+              "expr": "region_group_num_in_database{type=\"SchemaRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "SeriesSlot Count",
+          "title": "SchemaRegionGroup Count",
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "description": "The number of SchemaRegionGroups in each Database",
+          "description": "The number of DataRegionGroups in each DataNode",
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 10,
+                "fillOpacity": 4,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -1803,8 +2077,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1813,11 +2086,11 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 12,
-            "x": 0,
-            "y": 22
+            "w": 8,
+            "x": 8,
+            "y": 3
           },
-          "id": 105,
+          "id": 107,
           "options": {
             "legend": {
               "calcs": [
@@ -1839,34 +2112,30 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_group_num_in_database{type=\"SchemaRegion\"}",
+              "exemplar": true,
+              "expr": "region_group_num_in_database{type=\"DataRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "SchemaRegionGroup Count",
+          "title": "DataRegionGroup Count",
           "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "description": "The number of DataRegionGroups in each DataNode",
+          "description": "The number of SeriesSlots in each Database",
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
                 "drawStyle": "line",
-                "fillOpacity": 10,
+                "fillOpacity": 4,
                 "gradientMode": "none",
                 "hideFrom": {
                   "legend": false,
@@ -1894,8 +2163,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -1904,11 +2172,11 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 12,
-            "x": 12,
-            "y": 22
+            "w": 8,
+            "x": 16,
+            "y": 3
           },
-          "id": 107,
+          "id": 103,
           "options": {
             "legend": {
               "calcs": [
@@ -1930,13 +2198,15 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_group_num_in_database{type=\"DataRegion\"}",
+              "exemplar": true,
+              "expr": "series_slot_num_in_database{instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "DataRegionGroup Count",
+          "title": "SeriesSlot Count",
           "type": "timeseries"
         }
       ],
@@ -1949,56 +2219,139 @@
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 16
+        "y": 3
       },
       "id": 81,
       "panels": [
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "description": "The number of cluster SchemaRegions",
+          "description": "The status history of cluster DataRegions",
           "fieldConfig": {
             "defaults": {
               "color": {
-                "mode": "thresholds"
+                "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",
-                    "value": null
+                    "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": [
+                  {
+                    "id": "color",
+                    "value": {
+                      "fixedColor": "yellow",
+                      "mode": "fixed"
+                    }
                   }
                 ]
               }
-            },
-            "overrides": []
+            ]
           },
           "gridPos": {
             "h": 6,
-            "w": 6,
+            "w": 12,
             "x": 0,
-            "y": 17
+            "y": 4
           },
-          "id": 83,
+          "id": 91,
           "options": {
-            "colorMode": "value",
-            "graphMode": "area",
-            "justifyMode": "auto",
-            "orientation": "auto",
-            "reduceOptions": {
+            "legend": {
               "calcs": [
                 "lastNotNull"
               ],
-              "fields": "",
-              "values": false
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true
             },
-            "textMode": "auto"
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
           },
-          "pluginVersion": "9.3.6",
           "targets": [
             {
               "datasource": {
@@ -2006,46 +2359,77 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "sum by(type) (region_num{type=\"SchemaRegion\"})",
-              "legendFormat": "__auto",
+              "exemplar": true,
+              "expr": "region_num{type=\"DataRegion\", instance=\"$instance\"}",
+              "interval": "",
+              "legendFormat": "{{status}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "SchemaRegion Count",
-          "type": "stat"
+          "title": "DataRegion Status History",
+          "type": "timeseries"
         },
         {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "description": "The current status of cluster SchemaRegions",
+          "description": "The status history of cluster SchemaRegions",
           "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": []
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green"
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
             },
             "overrides": [
               {
                 "matcher": {
                   "id": "byName",
-                  "options": "Running"
+                  "options": "ReadOnly"
                 },
                 "properties": [
                   {
                     "id": "color",
                     "value": {
-                      "fixedColor": "green",
+                      "fixedColor": "orange",
                       "mode": "fixed"
                     }
                   }
@@ -2054,13 +2438,13 @@
               {
                 "matcher": {
                   "id": "byName",
-                  "options": "ReadOnly"
+                  "options": "Removing"
                 },
                 "properties": [
                   {
                     "id": "color",
                     "value": {
-                      "fixedColor": "orange",
+                      "fixedColor": "red",
                       "mode": "fixed"
                     }
                   }
@@ -2069,13 +2453,13 @@
               {
                 "matcher": {
                   "id": "byName",
-                  "options": "Removing"
+                  "options": "Running"
                 },
                 "properties": [
                   {
                     "id": "color",
                     "value": {
-                      "fixedColor": "red",
+                      "fixedColor": "green",
                       "mode": "fixed"
                     }
                   }
@@ -2100,27 +2484,19 @@
           },
           "gridPos": {
             "h": 6,
-            "w": 6,
-            "x": 6,
-            "y": 17
+            "w": 12,
+            "x": 12,
+            "y": 4
           },
-          "id": 87,
+          "id": 89,
           "options": {
             "legend": {
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "values": [
-                "value"
-              ]
-            },
-            "pieType": "pie",
-            "reduceOptions": {
               "calcs": [
                 "lastNotNull"
               ],
-              "fields": "",
-              "values": false
+              "displayMode": "table",
+              "placement": "right",
+              "showLegend": true
             },
             "tooltip": {
               "mode": "single",
@@ -2134,378 +2510,741 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "builder",
-              "expr": "region_num{type=\"SchemaRegion\"}",
+              "exemplar": true,
+              "expr": "region_num{type=\"SchemaRegion\", instance=\"$instance\"}",
+              "interval": "",
               "legendFormat": "{{status}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "SchemaRegion Current Status",
-          "type": "piechart"
+          "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}"
           },
-          "description": "The status history of cluster SchemaRegions",
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
-                "axisLabel": "",
-                "axisPlacement": "auto",
-                "barAlignment": 0,
-                "drawStyle": "line",
-                "fillOpacity": 10,
-                "gradientMode": "none",
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
-                },
-                "lineInterpolation": "linear",
-                "lineWidth": 1,
-                "pointSize": 5,
-                "scaleDistribution": {
-                  "type": "linear"
-                },
-                "showPoints": "never",
-                "spanNulls": false,
-                "stacking": {
-                  "group": "A",
-                  "mode": "none"
-                },
-                "thresholdsStyle": {
-                  "mode": "off"
+          "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"
                 }
               },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green",
-                    "value": null
-                  },
-                  {
-                    "color": "red",
-                    "value": 80
-                  }
-                ]
-              }
+              "type": "special"
             },
-            "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"
-                    }
-                  }
-                ]
+            {
+              "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": 17
+          "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": 89,
-          "options": {
-            "legend": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true
-            },
-            "tooltip": {
-              "mode": "single",
-              "sort": "none"
-            }
+          "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}"
           },
-          "targets": [
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "builder",
-              "expr": "region_num{type=\"SchemaRegion\"}",
-              "legendFormat": "{{status}}",
-              "range": true,
-              "refId": "A"
-            }
-          ],
-          "title": "SchemaRegion Status History",
-          "type": "timeseries"
+          "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}"
           },
-          "description": "The number of cluster DataRegions",
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "thresholds"
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green",
-                    "value": null
-                  }
-                ]
-              }
-            },
-            "overrides": []
+          "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}"
           },
-          "gridPos": {
-            "h": 6,
-            "w": 6,
-            "x": 0,
-            "y": 23
+          "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"
           },
-          "id": 85,
-          "options": {
-            "colorMode": "value",
-            "graphMode": "area",
-            "justifyMode": "auto",
-            "orientation": "auto",
-            "reduceOptions": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "fields": "",
-              "values": false
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 10,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
             },
-            "textMode": "auto"
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "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": "builder",
-              "expr": "sum by (type) (region_num{type=\"DataRegion\"})",
-              "legendFormat": "__auto",
-              "range": true,
-              "refId": "A"
+              "type": "special"
+            },
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
+              },
+              "type": "special"
             }
           ],
-          "title": "DataRegion Count",
-          "type": "stat"
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "s"
+        },
+        "overrides": [
+          {
+            "__systemRef": "hideSeriesFrom",
+            "matcher": {
+              "id": "byNames",
+              "options": {
+                "mode": "exclude",
+                "names": [
+                  "writeLocally"
+                ],
+                "prefix": "All except:",
+                "readOnly": true
+              }
+            },
+            "properties": [
+              {
+                "id": "custom.hideFrom",
+                "value": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": true
+                }
+              }
+            ]
+          }
+        ]
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 8,
+        "x": 0,
+        "y": 14
+      },
+      "id": 113,
+      "options": {
+        "legend": {
+          "calcs": [
+            "mean"
+          ],
+          "displayMode": "table",
+          "placement": "right"
         },
+        "tooltip": {
+          "mode": "multi",
+          "sort": "desc"
+        }
+      },
+      "targets": [
         {
           "datasource": {
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "The current status of cluster DataRegions",
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "palette-classic"
-              },
-              "custom": {
-                "hideFrom": {
-                  "legend": false,
-                  "tooltip": false,
-                  "viz": false
+          "exemplar": true,
+          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeLocally\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeLocally\"}[1m])",
+          "interval": "",
+          "legendFormat": "{{stage}}",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "exemplar": true,
+          "expr": "rate(ConfigRegion_ratis_consensus_write_seconds_sum{instance=\"$instance\", stage=\"writeRemotely\"}[1m]) / rate(ConfigRegion_ratis_consensus_write_seconds_count{instance=\"$instance\", stage=\"writeRemotely\"}[1m])",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "{{stage}}",
+          "refId": "B"
+        }
+      ],
+      "title": "Remote / Local Write Time",
+      "type": "timeseries"
+    },
+    {
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 10,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "never",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [
+            {
+              "options": {
+                "match": "empty",
+                "result": {
+                  "index": 0,
+                  "text": "0"
                 }
               },
-              "mappings": []
+              "type": "special"
             },
-            "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"
-                    }
-                  }
-                ]
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
               },
+              "type": "special"
+            }
+          ],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
               {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Removing"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "red",
-                      "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": 6,
-            "x": 6,
-            "y": 23
+          "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": 93,
-          "options": {
-            "legend": {
-              "displayMode": "table",
-              "placement": "right",
-              "showLegend": true,
-              "values": [
-                "value"
-              ]
+          "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
             },
-            "pieType": "pie",
-            "reduceOptions": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "fields": "",
-              "values": false
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
             },
-            "tooltip": {
-              "mode": "single",
-              "sort": "none"
+            "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": "builder",
-              "expr": "region_num{type=\"DataRegion\"}",
-              "legendFormat": "{{status}}",
-              "range": true,
-              "refId": "A"
+              "type": "special"
+            },
+            {
+              "options": {
+                "match": "null+nan",
+                "result": {
+                  "index": 1,
+                  "text": "0"
+                }
+              },
+              "type": "special"
             }
           ],
-          "title": "DataRegion Current Status",
-          "type": "piechart"
+          "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}"
           },
-          "description": "The status history of cluster DataRegions",
+          "exemplar": true,
+          "expr": "ConfigRegion_ratis_log_worker_openSegmentSizeInBytes {instance=\"$instance\"}",
+          "hide": false,
+          "interval": "",
+          "legendFormat": "openSegmentSizeInBytes",
+          "refId": "B"
+        }
+      ],
+      "title": "RatisConsensus Memory",
+      "type": "timeseries"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 23
+      },
+      "id": 13,
+      "panels": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
+                "axisSoftMax": 0,
+                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 10,
@@ -2533,97 +3272,34 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "absolute",
+                "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
-                  }
-                ]
-              }
-            },
-            "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"
-                    }
+                    "color": "green"
                   }
                 ]
               },
-              {
-                "matcher": {
-                  "id": "byName",
-                  "options": "Unknown"
-                },
-                "properties": [
-                  {
-                    "id": "color",
-                    "value": {
-                      "fixedColor": "yellow",
-                      "mode": "fixed"
-                    }
-                  }
-                ]
-              }
-            ]
+              "unit": "percent"
+            },
+            "overrides": []
           },
           "gridPos": {
-            "h": 6,
-            "w": 12,
-            "x": 12,
-            "y": 23
+            "h": 8,
+            "w": 8,
+            "x": 0,
+            "y": 5
           },
-          "id": 91,
+          "id": 15,
           "options": {
             "legend": {
-              "calcs": [
-                "lastNotNull"
-              ],
-              "displayMode": "table",
-              "placement": "right",
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom",
               "showLegend": true
             },
             "tooltip": {
-              "mode": "single",
-              "sort": "none"
+              "mode": "multi",
+              "sort": "desc"
             }
           },
           "targets": [
@@ -2632,100 +3308,29 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "builder",
-              "expr": "region_num{type=\"DataRegion\"}",
-              "legendFormat": "{{status}}",
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
+              "interval": "",
+              "legendFormat": "System CPU Load",
               "range": true,
               "refId": "A"
-            }
-          ],
-          "title": "DataRegion Status History",
-          "type": "timeseries"
-        }
-      ],
-      "title": "Region",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 17
-      },
-      "id": 13,
-      "panels": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
-          },
-          "fieldConfig": {
-            "defaults": {
-              "color": {
-                "mode": "thresholds"
-              },
-              "mappings": [],
-              "thresholds": {
-                "mode": "absolute",
-                "steps": [
-                  {
-                    "color": "green",
-                    "value": null
-                  },
-                  {
-                    "color": "red",
-                    "value": 80
-                  }
-                ]
-              },
-              "unit": "Core"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 18
-          },
-          "id": 14,
-          "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",
-              "expr": "sys_cpu_cores{instance=\"$instance\",name=\"system\"}",
+              "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
               "hide": false,
-              "legendFormat": "Process CPU Time",
+              "legendFormat": "Process CPU Load",
               "range": true,
               "refId": "B"
             }
           ],
-          "title": "CPU Core",
-          "type": "stat"
+          "title": "CPU Load",
+          "type": "timeseries"
         },
         {
           "datasource": {
@@ -2738,8 +3343,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "axisSoftMax": 0,
@@ -2774,12 +3377,11 @@
                 "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
-              "unit": "percent"
+              "unit": "ns"
             },
             "overrides": []
           },
@@ -2787,9 +3389,9 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 18
+            "y": 5
           },
-          "id": 15,
+          "id": 16,
           "options": {
             "legend": {
               "calcs": [],
@@ -2809,27 +3411,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "code",
-              "exemplar": true,
-              "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
-              "interval": "",
-              "legendFormat": "System CPU Load",
-              "range": true,
-              "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
+              "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
               "hide": false,
-              "legendFormat": "Process CPU Load",
+              "legendFormat": "Process CPU Time",
               "range": true,
               "refId": "B"
             }
           ],
-          "title": "CPU Load",
+          "title": "CPU Time Per Minute",
           "type": "timeseries"
         },
         {
@@ -2843,12 +3432,8 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
-                "axisSoftMax": 0,
-                "axisSoftMin": 0,
                 "barAlignment": 0,
                 "drawStyle": "line",
                 "fillOpacity": 10,
@@ -2876,25 +3461,70 @@
               },
               "mappings": [],
               "thresholds": {
-                "mode": "percentage",
+                "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "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"
+                    }
                   }
                 ]
-              },
-              "unit": "ns"
-            },
-            "overrides": []
+              }
+            ]
           },
           "gridPos": {
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 18
+            "y": 5
           },
-          "id": 16,
+          "id": 25,
           "options": {
             "legend": {
               "calcs": [],
@@ -2914,14 +3544,16 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "editorMode": "code",
-              "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
+              "exemplar": true,
+              "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
               "hide": false,
-              "legendFormat": "Process CPU Time",
+              "interval": "",
+              "legendFormat": "{{level}}",
               "range": true,
-              "refId": "B"
+              "refId": "C"
             }
           ],
-          "title": "CPU Time Per Minute",
+          "title": "Log Number Per Minute",
           "type": "timeseries"
         },
         {
@@ -2935,8 +3567,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -2969,8 +3599,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -2982,7 +3611,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 26
+            "y": 13
           },
           "id": 17,
           "options": {
@@ -3052,8 +3681,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3086,8 +3713,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3130,7 +3756,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 26
+            "y": 13
           },
           "id": 18,
           "options": {
@@ -3187,8 +3813,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3221,8 +3845,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   },
                   {
                     "color": "red",
@@ -3238,7 +3861,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 26
+            "y": 13
           },
           "id": 19,
           "options": {
@@ -3304,8 +3927,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3338,8 +3959,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -3350,7 +3970,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 34
+            "y": 21
           },
           "id": 20,
           "options": {
@@ -3406,8 +4026,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3440,8 +4058,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3453,7 +4070,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 34
+            "y": 21
           },
           "id": 21,
           "options": {
@@ -3507,8 +4124,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3541,8 +4156,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -3553,7 +4167,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 34
+            "y": 21
           },
           "id": 22,
           "options": {
@@ -3609,8 +4223,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3643,8 +4255,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3656,7 +4267,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 42
+            "y": 29
           },
           "id": 23,
           "options": {
@@ -3748,8 +4359,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3782,8 +4391,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -3795,7 +4403,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 42
+            "y": 29
           },
           "id": 24,
           "options": {
@@ -3839,8 +4447,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -3853,6 +4459,9 @@
                   "viz": false
                 },
                 "lineInterpolation": "linear",
+                "lineStyle": {
+                  "fill": "solid"
+                },
                 "lineWidth": 1,
                 "pointSize": 5,
                 "scaleDistribution": {
@@ -3873,68 +4482,20 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
-                  }
-                ]
-              },
-              "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"
-                    }
+                    "color": "green"
                   }
                 ]
               }
-            ]
+            },
+            "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 42
+            "y": 29
           },
-          "id": 25,
+          "id": 27,
           "options": {
             "legend": {
               "calcs": [],
@@ -3955,92 +4516,12 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
-              "hide": false,
+              "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
               "interval": "",
-              "legendFormat": "{{level}}",
+              "legendFormat": "Unloaded Number",
               "range": true,
-              "refId": "C"
-            }
-          ],
-          "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": 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",
-                    "value": null
-                  }
-                ]
-              },
-              "unit": "ms"
-            },
-            "overrides": []
-          },
-          "gridPos": {
-            "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 50
-          },
-          "id": 26,
-          "options": {
-            "legend": {
-              "calcs": [],
-              "displayMode": "list",
-              "placement": "bottom",
-              "showLegend": true
+              "refId": "A"
             },
-            "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
-            }
-          },
-          "targets": [
             {
               "datasource": {
                 "type": "prometheus",
@@ -4048,14 +4529,15 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
+              "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+              "hide": false,
               "interval": "",
-              "legendFormat": "compiler",
+              "legendFormat": "Loaded Number",
               "range": true,
-              "refId": "A"
+              "refId": "B"
             }
           ],
-          "title": "The Time Consumed of Compilation Per Minute",
+          "title": "The  Number Of Class",
           "type": "timeseries"
         },
         {
@@ -4063,14 +4545,13 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4083,9 +4564,6 @@
                   "viz": false
                 },
                 "lineInterpolation": "linear",
-                "lineStyle": {
-                  "fill": "solid"
-                },
                 "lineWidth": 1,
                 "pointSize": 5,
                 "scaleDistribution": {
@@ -4106,21 +4584,21 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "ms"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 8,
-            "x": 8,
-            "y": 50
+            "x": 0,
+            "y": 37
           },
-          "id": 27,
+          "id": 26,
           "options": {
             "legend": {
               "calcs": [],
@@ -4141,28 +4619,14 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
+              "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "Unloaded Number",
+              "legendFormat": "compiler",
               "range": true,
               "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Loaded Number",
-              "range": true,
-              "refId": "B"
             }
           ],
-          "title": "The  Number Of Class",
+          "title": "The Time Consumed of Compilation Per Minute",
           "type": "timeseries"
         }
       ],
@@ -4170,7 +4634,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -4182,13 +4646,13 @@
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 18
+        "y": 24
       },
       "id": 29,
       "panels": [
@@ -4203,8 +4667,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "axisSoftMax": 1,
@@ -4239,8 +4701,7 @@
                 "mode": "percentage",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   },
                   {
                     "color": "#EAB839",
@@ -4260,7 +4721,7 @@
             "h": 8,
             "w": 12,
             "x": 0,
-            "y": 59
+            "y": 6
           },
           "id": 31,
           "options": {
@@ -4308,8 +4769,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4342,8 +4801,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4355,7 +4813,7 @@
             "h": 8,
             "w": 12,
             "x": 12,
-            "y": 59
+            "y": 6
           },
           "id": 33,
           "options": {
@@ -4403,8 +4861,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4437,8 +4893,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4450,7 +4905,7 @@
             "h": 8,
             "w": 12,
             "x": 0,
-            "y": 67
+            "y": 14
           },
           "id": 35,
           "options": {
@@ -4497,8 +4952,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4531,8 +4984,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4544,7 +4996,7 @@
             "h": 8,
             "w": 12,
             "x": 12,
-            "y": 67
+            "y": 14
           },
           "id": 37,
           "options": {
@@ -4587,8 +5039,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4621,8 +5071,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4634,7 +5083,7 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 75
+            "y": 22
           },
           "id": 39,
           "options": {
@@ -4683,8 +5132,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4717,8 +5164,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4730,7 +5176,7 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 75
+            "y": 22
           },
           "id": 41,
           "options": {
@@ -4776,8 +5222,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4810,8 +5254,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               }
@@ -4822,7 +5265,7 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 75
+            "y": 22
           },
           "id": 43,
           "options": {
@@ -4870,8 +5313,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4904,8 +5345,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -4917,7 +5357,7 @@
             "h": 8,
             "w": 12,
             "x": 0,
-            "y": 83
+            "y": 30
           },
           "id": 47,
           "options": {
@@ -4961,8 +5401,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -4995,8 +5433,7 @@
                 "mode": "absolute",
                 "steps": [
                   {
-                    "color": "green",
-                    "value": null
+                    "color": "green"
                   }
                 ]
               },
@@ -5008,7 +5445,7 @@
             "h": 8,
             "w": 12,
             "x": 12,
-            "y": 83
+            "y": 30
           },
           "id": 45,
           "options": {
@@ -5048,7 +5485,7 @@
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
+            "uid": "${DS_PROMETHEUS}"
           },
           "refId": "A"
         }
@@ -5057,8 +5494,8 @@
       "type": "row"
     }
   ],
-  "refresh": "15s",
-  "schemaVersion": 37,
+  "refresh": false,
+  "schemaVersion": 35,
   "style": "dark",
   "tags": [
     "Apache-IoTDB",
@@ -5114,7 +5551,7 @@
     ]
   },
   "time": {
-    "from": "now-30m",
+    "from": "now-1h",
     "to": "now"
   },
   "timepicker": {
@@ -5128,8 +5565,8 @@
     ]
   },
   "timezone": "browser",
-  "title": "yesterday dashboard",
-  "uid": "TbEVYRw7T666",
-  "version": 1,
+  "title": "Apache IoTDB ConfigNode Dashboard",
+  "uid": "4WkTYkx4z",
+  "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 673e00ba31..bdbe5ec5a3 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-DataNode-Dashboard.json
@@ -9,7 +9,7 @@
       "pluginName": "Prometheus"
     }
   ],
-  "__elements": {},
+  "__elements": [],
   "__requires": [
     {
       "type": "panel",
@@ -21,7 +21,7 @@
       "type": "grafana",
       "id": "grafana",
       "name": "Grafana",
-      "version": "9.3.1"
+      "version": "8.4.2"
     },
     {
       "type": "panel",
@@ -75,18 +75,47 @@
   "fiscalYearStartMonth": 0,
   "graphTooltip": 1,
   "id": null,
+  "iteration": 1679118029653,
   "links": [],
   "liveNow": false,
   "panels": [
     {
+      "collapsed": false,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 0
+      },
+      "id": 1,
+      "panels": [],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "refId": "A"
+        }
+      ],
+      "title": "Overview",
+      "type": "row"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
       "fieldConfig": {
         "defaults": {
           "color": {
             "mode": "palette-classic"
           },
           "custom": {
-            "axisCenteredZero": false,
-            "axisColorMode": "text",
             "axisLabel": "",
             "axisPlacement": "auto",
             "barAlignment": 0,
@@ -121,10 +150,6 @@
               {
                 "color": "green",
                 "value": null
-              },
-              {
-                "color": "red",
-                "value": 80
               }
             ]
           }
@@ -133,11 +158,99 @@
       },
       "gridPos": {
         "h": 8,
-        "w": 12,
+        "w": 8,
         "x": 0,
-        "y": 0
+        "y": 1
+      },
+      "id": 2,
+      "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": "quantity{instance=~\"$instance\"}",
+          "interval": "",
+          "legendFormat": "{{name}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "The Number Of Entity",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
       },
-      "id": 231,
+      "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
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 8,
+        "y": 1
+      },
+      "id": 3,
       "options": {
         "legend": {
           "calcs": [],
@@ -146,26 +259,130 @@
           "showLegend": true
         },
         "tooltip": {
-          "mode": "single",
+          "mode": "multi",
           "sort": "none"
         }
       },
-      "title": "Panel Title",
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "editorMode": "code",
+          "exemplar": true,
+          "expr": "rate(quantity_total{instance=~\"$instance\"}[1m])",
+          "interval": "1m",
+          "legendFormat": "write point per second",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Write Point Per Second",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "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
+              }
+            ]
+          },
+          "unit": "bytes"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 16,
+        "y": 1
+      },
+      "id": 4,
+      "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": "mem{instance=~\"$instance\", name!=\"IoTConsensus\"}",
+          "interval": "",
+          "legendFormat": "{{name}}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Database Used Memory",
       "type": "timeseries"
     },
     {
       "collapsed": true,
       "datasource": {
         "type": "prometheus",
-        "uid": "lfuOmw1Vk"
+        "uid": "${DS_PROMETHEUS}"
       },
       "gridPos": {
         "h": 1,
         "w": 24,
         "x": 0,
-        "y": 8
+        "y": 9
       },
-      "id": 1,
+      "id": 5,
       "panels": [
         {
           "datasource": {
@@ -215,7 +432,8 @@
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "s"
             },
             "overrides": []
           },
@@ -223,9 +441,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 49
+            "y": 50
           },
-          "id": 2,
+          "id": 6,
           "options": {
             "legend": {
               "calcs": [],
@@ -244,16 +462,15 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "quantity{instance=~\"$instance\"}",
+              "expr": "statement_execution_seconds{instance=~\"$instance\", quantile=\"0.5\"}",
+              "hide": false,
               "interval": "",
-              "legendFormat": "{{name}}",
-              "range": true,
+              "legendFormat": "{{type}}/{{interface}}",
               "refId": "A"
             }
           ],
-          "title": "The Number Of Entity",
+          "title": "The Time Consumed Of Operation (50%)",
           "type": "timeseries"
         },
         {
@@ -304,7 +521,8 @@
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "s"
             },
             "overrides": []
           },
@@ -312,9 +530,9 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 49
+            "y": 50
           },
-          "id": 3,
+          "id": 55,
           "options": {
             "legend": {
               "calcs": [],
@@ -324,7 +542,7 @@
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "none"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -333,16 +551,15 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "rate(quantity_total{instance=~\"$instance\"}[1m])",
-              "interval": "1m",
-              "legendFormat": "write point per second",
-              "range": true,
+              "expr": "statement_execution_seconds{instance=~\"$instance\", quantile=\"0.75\"}",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "{{type}}/{{interface}}",
               "refId": "A"
             }
           ],
-          "title": "Write Point Per Second",
+          "title": "The Time Consumed Of Operation (75%)",
           "type": "timeseries"
         },
         {
@@ -394,7 +611,7 @@
                   }
                 ]
               },
-              "unit": "bytes"
+              "unit": "s"
             },
             "overrides": []
           },
@@ -402,9 +619,9 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 49
+            "y": 50
           },
-          "id": 4,
+          "id": 56,
           "options": {
             "legend": {
               "calcs": [],
@@ -423,49 +640,21 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "mem{instance=~\"$instance\", name!=\"IoTConsensus\"}",
+              "expr": "statement_execution_seconds{instance=~\"$instance\", quantile=\"1.0\"}",
+              "hide": false,
               "interval": "",
-              "legendFormat": "{{name}}",
-              "range": true,
+              "legendFormat": "{{type}}/{{interface}}",
               "refId": "A"
             }
           ],
-          "title": "Database Used Memory",
+          "title": "The Time Consumed Of Operation (100%)",
           "type": "timeseries"
-        }
-      ],
-      "targets": [
+        },
         {
           "datasource": {
             "type": "prometheus",
-            "uid": "lfuOmw1Vk"
-          },
-          "refId": "A"
-        }
-      ],
-      "title": "Overview",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 9
-      },
-      "id": 5,
-      "panels": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "${DS_PROMETHEUS}"
+            "uid": "${DS_PROMETHEUS}"
           },
           "fieldConfig": {
             "defaults": {
@@ -511,7 +700,7 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
@@ -519,9 +708,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 50
+            "y": 58
           },
-          "id": 6,
+          "id": 9,
           "options": {
             "legend": {
               "calcs": [],
@@ -540,15 +729,16 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "statement_execution_seconds{instance=~\"$instance\", quantile=\"0.5\"}",
-              "hide": false,
+              "expr": "rate(entry_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}/{{interface}}",
+              "legendFormat": "{{name}}",
+              "range": true,
               "refId": "A"
             }
           ],
-          "title": "The Time Consumed Of Operation (50%)",
+          "title": "The QPS Of Interface",
           "type": "timeseries"
         },
         {
@@ -608,9 +798,9 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 50
+            "y": 58
           },
-          "id": 55,
+          "id": 10,
           "options": {
             "legend": {
               "calcs": [],
@@ -629,15 +819,16 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "statement_execution_seconds{instance=~\"$instance\", quantile=\"0.75\"}",
-              "hide": false,
+              "expr": "avg(rate(entry_seconds_sum{instance=~\"$instance\"}[1m])/rate(entry_seconds_count{instance=~\"$instance\"}[1m])) by (name,instance)",
               "interval": "",
-              "legendFormat": "{{type}}/{{interface}}",
+              "legendFormat": "{{name}}",
+              "range": true,
               "refId": "A"
             }
           ],
-          "title": "The Time Consumed Of Operation (75%)",
+          "title": "The Time Consumed Of Interface",
           "type": "timeseries"
         },
         {
@@ -681,6 +872,8 @@
                 }
               },
               "mappings": [],
+              "max": 1,
+              "min": 0,
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
@@ -689,7 +882,7 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "percentunit"
             },
             "overrides": []
           },
@@ -697,9 +890,9 @@
             "h": 8,
             "w": 8,
             "x": 16,
-            "y": 50
+            "y": 58
           },
-          "id": 56,
+          "id": 11,
           "options": {
             "legend": {
               "calcs": [],
@@ -712,6 +905,7 @@
               "sort": "desc"
             }
           },
+          "pluginVersion": "8.4.2",
           "targets": [
             {
               "datasource": {
@@ -719,14 +913,26 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "statement_execution_seconds{instance=~\"$instance\", quantile=\"1.0\"}",
-              "hide": false,
+              "expr": "avg(cache_total{instance=~\"$instance\", type=\"hit\"}) by (name) / avg(cache_total{instance=~\"$instance\", type=\"all\"}) by (name)",
+              "format": "time_series",
               "interval": "",
-              "legendFormat": "{{type}}/{{interface}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "cache_hit{instance=~\"$instance\"} / 100",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "{{name}}",
+              "refId": "B"
             }
           ],
-          "title": "The Time Consumed Of Operation (100%)",
+          "title": "Cache Hit Rate",
           "type": "timeseries"
         },
         {
@@ -740,8 +946,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -786,9 +990,9 @@
             "h": 8,
             "w": 8,
             "x": 0,
-            "y": 58
+            "y": 66
           },
-          "id": 9,
+          "id": 12,
           "options": {
             "legend": {
               "calcs": [],
@@ -809,14 +1013,14 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "rate(entry_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "thrift_connections{instance=~\"$instance\"}",
               "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "The QPS Of Interface",
+          "title": "Thrift Connection",
           "type": "timeseries"
         },
         {
@@ -830,8 +1034,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -868,7 +1070,7 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
@@ -876,9 +1078,9 @@
             "h": 8,
             "w": 8,
             "x": 8,
-            "y": 58
+            "y": 66
           },
-          "id": 10,
+          "id": 13,
           "options": {
             "legend": {
               "calcs": [],
@@ -899,16 +1101,43 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "avg(rate(entry_seconds_sum{instance=~\"$instance\"}[1m])/rate(entry_seconds_count{instance=~\"$instance\"}[1m])) by (name,instance)",
+              "expr": "thrift_active_threads{instance=~\"$instance\"}",
               "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "The Time Consumed Of Interface",
+          "title": "Thrift Active Thread",
           "type": "timeseries"
-        },
+        }
+      ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "refId": "A"
+        }
+      ],
+      "title": "Interface",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 10
+      },
+      "id": 14,
+      "panels": [
         {
           "datasource": {
             "type": "prometheus",
@@ -950,8 +1179,6 @@
                 }
               },
               "mappings": [],
-              "max": 1,
-              "min": 0,
               "thresholds": {
                 "mode": "absolute",
                 "steps": [
@@ -959,18 +1186,17 @@
                     "color": "green"
                   }
                 ]
-              },
-              "unit": "percentunit"
+              }
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 58
+            "w": 12,
+            "x": 0,
+            "y": 11
           },
-          "id": 11,
+          "id": 15,
           "options": {
             "legend": {
               "calcs": [],
@@ -983,34 +1209,22 @@
               "sort": "desc"
             }
           },
-          "pluginVersion": "8.4.2",
           "targets": [
             {
               "datasource": {
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
+              "editorMode": "code",
               "exemplar": true,
-              "expr": "avg(cache_total{instance=~\"$instance\", type=\"hit\"}) by (name) / avg(cache_total{instance=~\"$instance\", type=\"all\"}) by (name)",
-              "format": "time_series",
+              "expr": "queue{instance=~\"$instance\"}",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{name}}-{{status}}",
+              "range": true,
               "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "cache_hit{instance=~\"$instance\"} / 100",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "{{name}}",
-              "refId": "B"
             }
           ],
-          "title": "Cache Hit Rate",
+          "title": "Task Number",
           "type": "timeseries"
         },
         {
@@ -1024,6 +1238,8 @@
                 "mode": "palette-classic"
               },
               "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1060,17 +1276,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
-            "x": 0,
-            "y": 66
+            "w": 12,
+            "x": 12,
+            "y": 11
           },
-          "id": 12,
+          "id": 16,
           "options": {
             "legend": {
               "calcs": [],
@@ -1091,14 +1307,14 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "thrift_connections{instance=~\"$instance\"}",
+              "expr": "rate(cost_task_seconds_sum{instance=~\"$instance\"}[1m])/rate(cost_task_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
               "legendFormat": "{{name}}",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "Thrift Connection",
+          "title": "The Time Consumed of Task",
           "type": "timeseries"
         },
         {
@@ -1112,6 +1328,8 @@
                 "mode": "palette-classic"
               },
               "custom": {
+                "axisCenteredZero": false,
+                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -1148,17 +1366,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "bytes"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 8,
-            "x": 8,
-            "y": 66
+            "x": 0,
+            "y": 19
           },
-          "id": 13,
+          "id": 17,
           "options": {
             "legend": {
               "calcs": [],
@@ -1179,43 +1397,28 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "thrift_active_threads{instance=~\"$instance\"}",
+              "expr": "sum(rate(data_written_total{instance=~\"$instance\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "Write",
               "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "expr": "sum(rate(data_read_total{instance=~\"$instance\"}[1m]))*60",
+              "hide": false,
+              "legendFormat": "Read",
+              "range": true,
+              "refId": "B"
             }
           ],
-          "title": "Thrift Active Thread",
+          "title": "Compaction Read And Write Per Minute",
           "type": "timeseries"
-        }
-      ],
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "lfuOmw1Vk"
-          },
-          "refId": "A"
-        }
-      ],
-      "title": "Interface",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 10
-      },
-      "id": 14,
-      "panels": [
+        },
         {
           "datasource": {
             "type": "prometheus",
@@ -1264,17 +1467,18 @@
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 0,
-            "y": 11
+            "w": 8,
+            "x": 8,
+            "y": 19
           },
-          "id": 15,
+          "id": 18,
           "options": {
             "legend": {
               "calcs": [],
@@ -1295,14 +1499,14 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "queue{instance=~\"$instance\"}",
+              "expr": "sum(rate(data_read_total{instance=~\"$instance\"}[1m]))*60 / sum(rate(data_written_total{instance=~\"$instance\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "{{name}}-{{status}}",
+              "legendFormat": "Ratio",
               "range": true,
               "refId": "A"
             }
           ],
-          "title": "Task Number",
+          "title": "Compaction R/W Ratio Per Minute",
           "type": "timeseries"
         },
         {
@@ -1354,17 +1558,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 12,
-            "y": 11
+            "w": 8,
+            "x": 16,
+            "y": 19
           },
-          "id": 16,
+          "id": 19,
           "options": {
             "legend": {
               "calcs": [],
@@ -1385,14 +1589,42 @@
               },
               "editorMode": "code",
               "exemplar": true,
-              "expr": "rate(cost_task_seconds_sum{instance=~\"$instance\"}[1m])/rate(cost_task_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "sum(rate(compaction_task_count{instance=\"$instance\", name = \"inner_seq\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "sequence",
               "range": true,
               "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sum(rate(compaction_task_count{instance=\"$instance\", name = \"inner_unseq\"}[1m]))*60",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "unsequence",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "editorMode": "code",
+              "exemplar": true,
+              "expr": "sum(rate(compaction_task_count{instance=\"$instance\", name = \"cross\"}[1m]))*60",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "cross",
+              "range": true,
+              "refId": "C"
             }
           ],
-          "title": "The Time Consumed of Task",
+          "title": "Compaction Number Per Minute",
           "type": "timeseries"
         },
         {
@@ -1443,18 +1675,17 @@
                     "color": "green"
                   }
                 ]
-              },
-              "unit": "bytes"
+              }
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
+            "w": 12,
             "x": 0,
-            "y": 19
+            "y": 27
           },
-          "id": 17,
+          "id": 169,
           "options": {
             "legend": {
               "calcs": [],
@@ -1463,8 +1694,8 @@
               "showLegend": true
             },
             "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
+              "mode": "single",
+              "sort": "none"
             }
           },
           "targets": [
@@ -1473,12 +1704,10 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(data_written_total{instance=~\"$instance\"}[1m]))*60",
+              "expr": "sum(rate(Compacted_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "Write",
-              "range": true,
+              "legendFormat": "Compacted Chunk Num Per Min",
               "refId": "A"
             },
             {
@@ -1486,15 +1715,39 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
-              "expr": "sum(rate(data_read_total{instance=~\"$instance\"}[1m]))*60",
+              "exemplar": true,
+              "expr": "sum(rate(Deserialized_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
               "hide": false,
-              "legendFormat": "Read",
-              "range": true,
+              "interval": "",
+              "legendFormat": "Deserialized Chunk Num Per Min",
               "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(rate(Directly_Flush_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Directly Flush Chunk Num Per Min",
+              "refId": "C"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "${DS_PROMETHEUS}"
+              },
+              "exemplar": true,
+              "expr": "sum(rate(Merged_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Merged Chunk Num Per Min",
+              "refId": "D"
             }
           ],
-          "title": "Compaction Read And Write Per Minute",
+          "title": "Compaction Process Chunk Status",
           "type": "timeseries"
         },
         {
@@ -1545,18 +1798,17 @@
                     "color": "green"
                   }
                 ]
-              },
-              "unit": "none"
+              }
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 19
+            "w": 12,
+            "x": 12,
+            "y": 27
           },
-          "id": 18,
+          "id": 171,
           "options": {
             "legend": {
               "calcs": [],
@@ -1565,8 +1817,8 @@
               "showLegend": true
             },
             "tooltip": {
-              "mode": "multi",
-              "sort": "desc"
+              "mode": "single",
+              "sort": "none"
             }
           },
           "targets": [
@@ -1575,18 +1827,43 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(data_read_total{instance=~\"$instance\"}[1m]))*60 / sum(rate(data_written_total{instance=~\"$instance\"}[1m]))*60",
+              "expr": "sum(rate(Compacted_Point_Num_total{instance=~\"$instance\"}[1m]))*60",
               "interval": "",
-              "legendFormat": "Ratio",
-              "range": true,
+              "legendFormat": "Compacted Point Num Per Min",
               "refId": "A"
             }
           ],
-          "title": "Compaction R/W Ratio Per Minute",
+          "title": "Compacted Point Num Per Minute",
           "type": "timeseries"
-        },
+        }
+      ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "refId": "A"
+        }
+      ],
+      "title": "Engine",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 11
+      },
+      "id": 58,
+      "panels": [
         {
           "datasource": {
             "type": "prometheus",
@@ -1636,17 +1913,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 19
+            "w": 6,
+            "x": 0,
+            "y": 12
           },
-          "id": 19,
+          "id": 80,
           "options": {
             "legend": {
               "calcs": [],
@@ -1665,44 +1942,14 @@
                 "type": "prometheus",
                 "uid": "${DS_PROMETHEUS}"
               },
-              "editorMode": "code",
               "exemplar": true,
-              "expr": "sum(rate(compaction_task_count{instance=\"$instance\", name = \"inner_seq\"}[1m]))*60",
+              "expr": "rate(query_plan_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(query_plan_cost_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "sequence",
-              "range": true,
+              "legendFormat": "{{stage}}",
               "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sum(rate(compaction_task_count{instance=\"$instance\", name = \"inner_unseq\"}[1m]))*60",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "unsequence",
-              "range": true,
-              "refId": "B"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "editorMode": "code",
-              "exemplar": true,
-              "expr": "sum(rate(compaction_task_count{instance=\"$instance\", name = \"cross\"}[1m]))*60",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "cross",
-              "range": true,
-              "refId": "C"
             }
           ],
-          "title": "Compaction Number Per Minute",
+          "title": "The time consumed of query plan stages(avg)",
           "type": "timeseries"
         },
         {
@@ -1753,17 +2000,18 @@
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
-            "x": 0,
-            "y": 27
+            "w": 6,
+            "x": 6,
+            "y": 12
           },
-          "id": 169,
+          "id": 60,
           "options": {
             "legend": {
               "calcs": [],
@@ -1772,8 +2020,8 @@
               "showLegend": true
             },
             "tooltip": {
-              "mode": "single",
-              "sort": "none"
+              "mode": "multi",
+              "sort": "desc"
             }
           },
           "targets": [
@@ -1783,49 +2031,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(Compacted_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
+              "expr": "query_plan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
               "interval": "",
-              "legendFormat": "Compacted Chunk Num Per Min",
+              "legendFormat": "{{stage}}",
               "refId": "A"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "sum(rate(Deserialized_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Deserialized Chunk Num Per Min",
-              "refId": "B"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "sum(rate(Directly_Flush_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Directly Flush Chunk Num Per Min",
-              "refId": "C"
-            },
-            {
-              "datasource": {
-                "type": "prometheus",
-                "uid": "${DS_PROMETHEUS}"
-              },
-              "exemplar": true,
-              "expr": "sum(rate(Merged_Chunk_Num_total{instance=~\"$instance\"}[1m]))*60",
-              "hide": false,
-              "interval": "",
-              "legendFormat": "Merged Chunk Num Per Min",
-              "refId": "D"
             }
           ],
-          "title": "Compaction Process Chunk Status",
+          "title": "The time consumed of query plan stages(50%)",
           "type": "timeseries"
         },
         {
@@ -1876,17 +2088,18 @@
                     "color": "green"
                   }
                 ]
-              }
+              },
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 12,
+            "w": 6,
             "x": 12,
-            "y": 27
+            "y": 12
           },
-          "id": 171,
+          "id": 61,
           "options": {
             "legend": {
               "calcs": [],
@@ -1895,8 +2108,8 @@
               "showLegend": true
             },
             "tooltip": {
-              "mode": "single",
-              "sort": "none"
+              "mode": "multi",
+              "sort": "desc"
             }
           },
           "targets": [
@@ -1906,42 +2119,15 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(Compacted_Point_Num_total{instance=~\"$instance\"}[1m]))*60",
+              "expr": "query_plan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
               "interval": "",
-              "legendFormat": "Compacted Point Num Per Min",
+              "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "Compacted Point Num Per Minute",
+          "title": "The time consumed of query plan stages(75%)",
           "type": "timeseries"
-        }
-      ],
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "lfuOmw1Vk"
-          },
-          "refId": "A"
-        }
-      ],
-      "title": "Engine",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 11
-      },
-      "id": 58,
-      "panels": [
+        },
         {
           "datasource": {
             "type": "prometheus",
@@ -1998,10 +2184,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 12
           },
-          "id": 80,
+          "id": 62,
           "options": {
             "legend": {
               "calcs": [],
@@ -2021,13 +2207,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(query_plan_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(query_plan_cost_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "query_plan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query plan stages(avg)",
+          "title": "The time consumed of query plan stages(100%)",
           "type": "timeseries"
         },
         {
@@ -2086,10 +2272,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 12
+            "x": 0,
+            "y": 20
           },
-          "id": 60,
+          "id": 63,
           "options": {
             "legend": {
               "calcs": [],
@@ -2109,13 +2295,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_plan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
+              "expr": "rate(dispatcher_seconds_sum{instance=~\"$instance\"}[1m]) / rate(dispatcher_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query plan stages(50%)",
+          "title": "The time consumed of plan dispatch stages(avg)",
           "type": "timeseries"
         },
         {
@@ -2174,10 +2360,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 12
+            "x": 6,
+            "y": 20
           },
-          "id": 61,
+          "id": 81,
           "options": {
             "legend": {
               "calcs": [],
@@ -2197,13 +2383,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_plan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
+              "expr": "dispatcher_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query plan stages(75%)",
+          "title": "The time consumed of plan dispatch stages(50%)",
           "type": "timeseries"
         },
         {
@@ -2262,10 +2448,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 12
+            "x": 12,
+            "y": 20
           },
-          "id": 62,
+          "id": 64,
           "options": {
             "legend": {
               "calcs": [],
@@ -2285,13 +2471,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_plan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
+              "expr": "dispatcher_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query plan stages(100%)",
+          "title": "The time consumed of plan dispatch stages(75%)",
           "type": "timeseries"
         },
         {
@@ -2350,10 +2536,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 20
           },
-          "id": 63,
+          "id": 65,
           "options": {
             "legend": {
               "calcs": [],
@@ -2373,13 +2559,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(dispatcher_seconds_sum{instance=~\"$instance\"}[1m]) / rate(dispatcher_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "dispatcher_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of plan dispatch stages(avg)",
+          "title": "The time consumed of plan dispatch stages(100%)",
           "type": "timeseries"
         },
         {
@@ -2438,10 +2624,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 20
+            "x": 0,
+            "y": 28
           },
-          "id": 81,
+          "id": 83,
           "options": {
             "legend": {
               "calcs": [],
@@ -2461,13 +2647,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "dispatcher_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
+              "expr": "rate(query_execution_seconds_sum{instance=~\"$instance\"}[1m]) / rate(query_execution_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of plan dispatch stages(50%)",
+          "title": "The time consumed of query execution stages(avg)",
           "type": "timeseries"
         },
         {
@@ -2526,10 +2712,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 20
+            "x": 6,
+            "y": 28
           },
-          "id": 64,
+          "id": 66,
           "options": {
             "legend": {
               "calcs": [],
@@ -2549,13 +2735,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "dispatcher_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
+              "expr": "query_execution_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of plan dispatch stages(75%)",
+          "title": "The time consumed of query execution stages(50%)",
           "type": "timeseries"
         },
         {
@@ -2614,10 +2800,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 20
+            "x": 12,
+            "y": 28
           },
-          "id": 65,
+          "id": 67,
           "options": {
             "legend": {
               "calcs": [],
@@ -2627,7 +2813,7 @@
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "desc"
+              "sort": "asc"
             }
           },
           "targets": [
@@ -2637,13 +2823,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "dispatcher_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
+              "expr": "query_execution_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of plan dispatch stages(100%)",
+          "title": "The time consumed of query execution stages(75%)",
           "type": "timeseries"
         },
         {
@@ -2702,10 +2888,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 28
           },
-          "id": 83,
+          "id": 68,
           "options": {
             "legend": {
               "calcs": [],
@@ -2715,7 +2901,7 @@
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "desc"
+              "sort": "asc"
             }
           },
           "targets": [
@@ -2725,13 +2911,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(query_execution_seconds_sum{instance=~\"$instance\"}[1m]) / rate(query_execution_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "query_execution_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query execution stages(avg)",
+          "title": "The time consumed of query execution stages(100%)",
           "type": "timeseries"
         },
         {
@@ -2790,10 +2976,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 28
+            "x": 0,
+            "y": 36
           },
-          "id": 66,
+          "id": 82,
           "options": {
             "legend": {
               "calcs": [],
@@ -2813,13 +2999,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_execution_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
+              "expr": "rate(operator_execution_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(operator_execution_cost_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "{{stage}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query execution stages(50%)",
+          "title": "The time consumed of operator execution stages(avg)",
           "type": "timeseries"
         },
         {
@@ -2878,10 +3064,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 28
+            "x": 6,
+            "y": 36
           },
-          "id": 67,
+          "id": 69,
           "options": {
             "legend": {
               "calcs": [],
@@ -2901,13 +3087,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_execution_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
+              "expr": "operator_execution_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
               "interval": "",
-              "legendFormat": "{{stage}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query execution stages(75%)",
+          "title": "The time consumed of operator execution(50%)",
           "type": "timeseries"
         },
         {
@@ -2966,10 +3152,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 28
+            "x": 12,
+            "y": 36
           },
-          "id": 68,
+          "id": 70,
           "options": {
             "legend": {
               "calcs": [],
@@ -2979,7 +3165,7 @@
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -2989,13 +3175,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_execution_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
+              "expr": "operator_execution_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
               "interval": "",
-              "legendFormat": "{{stage}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query execution stages(100%)",
+          "title": "The time consumed of operator execution(75%)",
           "type": "timeseries"
         },
         {
@@ -3054,10 +3240,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 36
           },
-          "id": 82,
+          "id": 71,
           "options": {
             "legend": {
               "calcs": [],
@@ -3077,13 +3263,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(operator_execution_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(operator_execution_cost_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "operator_execution_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
               "interval": "",
               "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of operator execution stages(avg)",
+          "title": "The time consumed of operator execution(100%)",
           "type": "timeseries"
         },
         {
@@ -3142,10 +3328,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 36
+            "x": 0,
+            "y": 44
           },
-          "id": 69,
+          "id": 72,
           "options": {
             "legend": {
               "calcs": [],
@@ -3155,7 +3341,7 @@
             },
             "tooltip": {
               "mode": "multi",
-              "sort": "asc"
+              "sort": "desc"
             }
           },
           "targets": [
@@ -3165,13 +3351,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "operator_execution_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
+              "expr": "rate(aggregation_seconds_sum{instance=~\"$instance\"}[1m]) / rate(aggregation_seconds_count{instance=~\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of operator execution(50%)",
+          "title": "The time consumed of query aggregation(avg)",
           "type": "timeseries"
         },
         {
@@ -3230,10 +3416,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 36
+            "x": 6,
+            "y": 44
           },
-          "id": 70,
+          "id": 84,
           "options": {
             "legend": {
               "calcs": [],
@@ -3253,13 +3439,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "operator_execution_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
+              "expr": "aggregation_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of operator execution(75%)",
+          "title": "The time consumed of query aggregation(50%)",
           "type": "timeseries"
         },
         {
@@ -3318,10 +3504,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 36
+            "x": 12,
+            "y": 44
           },
-          "id": 71,
+          "id": 73,
           "options": {
             "legend": {
               "calcs": [],
@@ -3341,13 +3527,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "operator_execution_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
+              "expr": "aggregation_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of operator execution(100%)",
+          "title": "The time consumed of query aggregation(75%)",
           "type": "timeseries"
         },
         {
@@ -3406,10 +3592,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 44
           },
-          "id": 72,
+          "id": 74,
           "options": {
             "legend": {
               "calcs": [],
@@ -3429,13 +3615,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(aggregation_seconds_sum{instance=~\"$instance\"}[1m]) / rate(aggregation_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "aggregation_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
               "interval": "",
               "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query aggregation(avg)",
+          "title": "The time consumed of query aggregation(100%)",
           "type": "timeseries"
         },
         {
@@ -3494,10 +3680,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 44
+            "x": 0,
+            "y": 52
           },
-          "id": 84,
+          "id": 131,
           "options": {
             "legend": {
               "calcs": [],
@@ -3517,13 +3703,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "aggregation_seconds{instance=~\"${instance}\", quantile=\"0.5\"}",
+              "expr": "sum(rate(series_scan_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\"}[1m])) by (stage)",
+              "format": "time_series",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query aggregation(50%)",
+          "title": "The time consumed of query scan(avg)",
           "type": "timeseries"
         },
         {
@@ -3582,10 +3769,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 44
+            "x": 6,
+            "y": 52
           },
-          "id": 73,
+          "id": 132,
           "options": {
             "legend": {
               "calcs": [],
@@ -3605,13 +3792,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "aggregation_seconds{instance=~\"${instance}\", quantile=\"0.75\"}",
+              "expr": "sum(rate(series_scan_cost_seconds{instance=~\"$instance\", quantile=\"0.5\"}[1m])) by (stage)",
+              "format": "time_series",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query aggregation(75%)",
+          "title": "The time consumed of query scan(50%)",
           "type": "timeseries"
         },
         {
@@ -3670,10 +3858,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 44
+            "x": 12,
+            "y": 52
           },
-          "id": 74,
+          "id": 133,
           "options": {
             "legend": {
               "calcs": [],
@@ -3693,13 +3881,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "aggregation_seconds{instance=~\"${instance}\", quantile=\"1.0\"}",
+              "expr": "sum(rate(series_scan_cost_seconds{instance=~\"$instance\", quantile=\"0.75\"}[1m])) by (stage)",
+              "format": "time_series",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query aggregation(100%)",
+          "title": "The time consumed of query scan(75%)",
           "type": "timeseries"
         },
         {
@@ -3707,6 +3896,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -3758,10 +3948,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 52
           },
-          "id": 131,
+          "id": 134,
           "options": {
             "legend": {
               "calcs": [],
@@ -3781,14 +3971,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(series_scan_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\"}[1m])) by (stage)",
+              "expr": "sum(rate(series_scan_cost_seconds{instance=~\"$instance\", quantile=\"1.0\"}[1m])) by (stage)",
               "format": "time_series",
               "interval": "",
               "legendFormat": "{{stage}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query scan(avg)",
+          "title": "The time consumed of query scan(100%)",
           "type": "timeseries"
         },
         {
@@ -3840,17 +4030,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 52
+            "x": 0,
+            "y": 60
           },
-          "id": 132,
+          "id": 76,
           "options": {
             "legend": {
               "calcs": [],
@@ -3870,14 +4060,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(series_scan_cost_seconds{instance=~\"$instance\", quantile=\"0.5\"}[1m])) by (stage)",
-              "format": "time_series",
+              "expr": "rate(query_resource_sum{instance=~\"$instance\"}[1m]) / rate(query_resource_count{instance=~\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "{{stage}}",
+              "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query scan(50%)",
+          "title": "The usage of query resource(avg)",
           "type": "timeseries"
         },
         {
@@ -3929,17 +4118,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 52
+            "x": 6,
+            "y": 60
           },
-          "id": 133,
+          "id": 128,
           "options": {
             "legend": {
               "calcs": [],
@@ -3959,14 +4148,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(series_scan_cost_seconds{instance=~\"$instance\", quantile=\"0.75\"}[1m])) by (stage)",
-              "format": "time_series",
+              "expr": "query_resource{instance=~\"${instance}\", quantile=\"0.5\"}",
               "interval": "",
-              "legendFormat": "{{stage}}",
+              "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query scan(75%)",
+          "title": "The usage of query resource(50%)",
           "type": "timeseries"
         },
         {
@@ -3974,7 +4162,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -4019,17 +4206,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 52
+            "x": 12,
+            "y": 60
           },
-          "id": 134,
+          "id": 129,
           "options": {
             "legend": {
               "calcs": [],
@@ -4049,14 +4236,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(series_scan_cost_seconds{instance=~\"$instance\", quantile=\"1.0\"}[1m])) by (stage)",
-              "format": "time_series",
+              "expr": "query_resource{instance=~\"${instance}\", quantile=\"0.75\"}",
               "interval": "",
-              "legendFormat": "{{stage}}",
+              "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query scan(100%)",
+          "title": "The usage of query resource(75%)",
           "type": "timeseries"
         },
         {
@@ -4115,10 +4301,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 60
           },
-          "id": 76,
+          "id": 130,
           "options": {
             "legend": {
               "calcs": [],
@@ -4138,13 +4324,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(query_resource_sum{instance=~\"$instance\"}[1m]) / rate(query_resource_count{instance=~\"$instance\"}[1m])",
+              "expr": "query_resource{instance=~\"${instance}\", quantile=\"1.0\"}",
               "interval": "",
               "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The usage of query resource(avg)",
+          "title": "The usage of query resource(100%)",
           "type": "timeseries"
         },
         {
@@ -4196,17 +4382,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 60
+            "x": 0,
+            "y": 68
           },
-          "id": 128,
+          "id": 135,
           "options": {
             "legend": {
               "calcs": [],
@@ -4226,13 +4412,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_resource{instance=~\"${instance}\", quantile=\"0.5\"}",
+              "expr": "sum(rate(data_exchange_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(data_exchange_cost_seconds_count{instance=~\"$instance\"}[1m])) by (operation)",
               "interval": "",
-              "legendFormat": "{{type}}",
+              "legendFormat": "{{operation}}",
               "refId": "A"
             }
           ],
-          "title": "The usage of query resource(50%)",
+          "title": "The time consumed  of query data exchange(avg)",
           "type": "timeseries"
         },
         {
@@ -4284,17 +4470,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 60
+            "x": 6,
+            "y": 68
           },
-          "id": 129,
+          "id": 136,
           "options": {
             "legend": {
               "calcs": [],
@@ -4314,13 +4500,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_resource{instance=~\"${instance}\", quantile=\"0.75\"}",
+              "expr": "sum(rate(data_exchange_cost_seconds{instance=~\"$instance\", quantile=\"0.5\"}[1m])) by (operation)",
               "interval": "",
-              "legendFormat": "{{type}}",
+              "legendFormat": "{{operation}}",
               "refId": "A"
             }
           ],
-          "title": "The usage of query resource(75%)",
+          "title": "The time consumed  of query data exchange(50%)",
           "type": "timeseries"
         },
         {
@@ -4372,17 +4558,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 60
+            "x": 12,
+            "y": 68
           },
-          "id": 130,
+          "id": 137,
           "options": {
             "legend": {
               "calcs": [],
@@ -4402,13 +4588,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "query_resource{instance=~\"${instance}\", quantile=\"1.0\"}",
+              "expr": "sum(rate(data_exchange_cost_seconds{instance=~\"$instance\", quantile=\"0.75\"}[1m])) by (operation)",
               "interval": "",
-              "legendFormat": "{{type}}",
+              "legendFormat": "{{operation}}",
               "refId": "A"
             }
           ],
-          "title": "The usage of query resource(100%)",
+          "title": "The time consumed  of query data exchange(75%)",
           "type": "timeseries"
         },
         {
@@ -4467,10 +4653,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 68
           },
-          "id": 135,
+          "id": 138,
           "options": {
             "legend": {
               "calcs": [],
@@ -4490,13 +4676,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(data_exchange_cost_seconds_sum{instance=~\"$instance\"}[1m]) / rate(data_exchange_cost_seconds_count{instance=~\"$instance\"}[1m])) by (operation)",
+              "expr": "sum(rate(data_exchange_cost_seconds{instance=~\"$instance\", quantile=\"1.0\"}[1m])) by (operation)",
               "interval": "",
               "legendFormat": "{{operation}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed  of query data exchange(avg)",
+          "title": "The time consumed  of query data exchange(100%)",
           "type": "timeseries"
         },
         {
@@ -4548,17 +4734,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 6,
-            "x": 6,
-            "y": 68
+            "w": 8,
+            "x": 0,
+            "y": 76
           },
-          "id": 136,
+          "id": 139,
           "options": {
             "legend": {
               "calcs": [],
@@ -4578,13 +4764,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(data_exchange_cost_seconds{instance=~\"$instance\", quantile=\"0.5\"}[1m])) by (operation)",
+              "expr": "rate(data_exchange_count_sum{instance=~\"$instance\"}[1m]) / rate(data_exchange_count_count{instance=~\"$instance\"}[1m])",
               "interval": "",
-              "legendFormat": "{{operation}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed  of query data exchange(50%)",
+          "title": "The count of Data Exchange(avg)",
           "type": "timeseries"
         },
         {
@@ -4636,17 +4822,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 6,
-            "x": 12,
-            "y": 68
+            "w": 8,
+            "x": 8,
+            "y": 76
           },
-          "id": 137,
+          "id": 140,
           "options": {
             "legend": {
               "calcs": [],
@@ -4666,13 +4852,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(data_exchange_cost_seconds{instance=~\"$instance\", quantile=\"0.75\"}[1m])) by (operation)",
+              "expr": "data_exchange_count{instance=~\"$instance\"}",
               "interval": "",
-              "legendFormat": "{{operation}}",
+              "legendFormat": "{{quantile}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed  of query data exchange(75%)",
+          "title": "The count of Data Exchange",
           "type": "timeseries"
         },
         {
@@ -4724,17 +4910,17 @@
                   }
                 ]
               },
-              "unit": "s"
+              "unit": "none"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 6,
-            "x": 18,
-            "y": 68
+            "w": 8,
+            "x": 16,
+            "y": 76
           },
-          "id": 138,
+          "id": 167,
           "options": {
             "legend": {
               "calcs": [],
@@ -4754,13 +4940,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "sum(rate(data_exchange_cost_seconds{instance=~\"$instance\", quantile=\"1.0\"}[1m])) by (operation)",
+              "expr": "driver_scheduler{instance=~\"$instance\"}",
+              "format": "time_series",
               "interval": "",
-              "legendFormat": "{{operation}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed  of query data exchange(100%)",
+          "title": "The number of query queue",
           "type": "timeseries"
         },
         {
@@ -4812,17 +4999,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
+            "w": 6,
             "x": 0,
-            "y": 76
+            "y": 84
           },
-          "id": 139,
+          "id": 163,
           "options": {
             "legend": {
               "calcs": [],
@@ -4842,13 +5029,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(data_exchange_count_sum{instance=~\"$instance\"}[1m]) / rate(data_exchange_count_count{instance=~\"$instance\"}[1m])",
+              "expr": "rate(driver_scheduler_seconds_sum{instance=~\"$instance\"}[1m]) / rate(driver_scheduler_seconds_count{instance=~\"$instance\"}[1m])",
+              "format": "time_series",
               "interval": "",
               "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The count of Data Exchange(avg)",
+          "title": "The time consumed of query schedule time(avg)",
           "type": "timeseries"
         },
         {
@@ -4900,17 +5088,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
-            "x": 8,
-            "y": 76
+            "w": 6,
+            "x": 6,
+            "y": 84
           },
-          "id": 140,
+          "id": 164,
           "options": {
             "legend": {
               "calcs": [],
@@ -4930,13 +5118,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "data_exchange_count{instance=~\"$instance\"}",
+              "expr": "driver_scheduler_seconds{instance=~\"$instance\", quantile=\"0.5\"}",
+              "format": "time_series",
               "interval": "",
-              "legendFormat": "{{quantile}}",
+              "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The count of Data Exchange",
+          "title": "The time consumed of query schedule time(50%)",
           "type": "timeseries"
         },
         {
@@ -4988,17 +5177,17 @@
                   }
                 ]
               },
-              "unit": "none"
+              "unit": "s"
             },
             "overrides": []
           },
           "gridPos": {
             "h": 8,
-            "w": 8,
-            "x": 16,
-            "y": 76
+            "w": 6,
+            "x": 12,
+            "y": 84
           },
-          "id": 167,
+          "id": 165,
           "options": {
             "legend": {
               "calcs": [],
@@ -5018,14 +5207,14 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "driver_scheduler{instance=~\"$instance\"}",
+              "expr": "driver_scheduler_seconds{instance=~\"$instance\", quantile=\"0.75\"}",
               "format": "time_series",
               "interval": "",
               "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The number of query queue",
+          "title": "The time consumed of query schedule time(75%)",
           "type": "timeseries"
         },
         {
@@ -5084,10 +5273,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 84
           },
-          "id": 163,
+          "id": 166,
           "options": {
             "legend": {
               "calcs": [],
@@ -5107,16 +5296,43 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(driver_scheduler_seconds_sum{instance=~\"$instance\"}[1m]) / rate(driver_scheduler_seconds_count{instance=~\"$instance\"}[1m])",
+              "expr": "driver_scheduler_seconds{instance=~\"$instance\", quantile=\"1.0\"}",
               "format": "time_series",
               "interval": "",
               "legendFormat": "{{name}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query schedule time(avg)",
+          "title": "The time consumed of query schedule time(100%)",
           "type": "timeseries"
-        },
+        }
+      ],
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "${DS_PROMETHEUS}"
+          },
+          "refId": "A"
+        }
+      ],
+      "title": "Query Engine",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "${DS_PROMETHEUS}"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 12
+      },
+      "id": 100,
+      "panels": [
         {
           "datasource": {
             "type": "prometheus",
@@ -5173,10 +5389,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 84
+            "x": 0,
+            "y": 13
           },
-          "id": 164,
+          "id": 75,
           "options": {
             "legend": {
               "calcs": [],
@@ -5196,14 +5412,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "driver_scheduler_seconds{instance=~\"$instance\", quantile=\"0.5\"}",
-              "format": "time_series",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"load_timeseries_metadata\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"load_timeseries_metadata\"}[1m])",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query schedule time(50%)",
+          "title": "The time consumed of load timeseries metadata(avg)",
           "type": "timeseries"
         },
         {
@@ -5262,10 +5477,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 84
+            "x": 6,
+            "y": 13
           },
-          "id": 165,
+          "id": 85,
           "options": {
             "legend": {
               "calcs": [],
@@ -5285,14 +5500,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "driver_scheduler_seconds{instance=~\"$instance\", quantile=\"0.75\"}",
-              "format": "time_series",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"load_timeseries_metadata\"}",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query schedule time(75%)",
+          "title": "The time consumed of load timeseries metadata(50%)",
           "type": "timeseries"
         },
         {
@@ -5351,10 +5565,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 84
+            "x": 12,
+            "y": 13
           },
-          "id": 166,
+          "id": 77,
           "options": {
             "legend": {
               "calcs": [],
@@ -5374,43 +5588,15 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "driver_scheduler_seconds{instance=~\"$instance\", quantile=\"1.0\"}",
-              "format": "time_series",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"load_timeseries_metadata\"}",
               "interval": "",
-              "legendFormat": "{{name}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of query schedule time(100%)",
+          "title": "The time consumed of load timeseries metadata(75%)",
           "type": "timeseries"
-        }
-      ],
-      "targets": [
-        {
-          "datasource": {
-            "type": "prometheus",
-            "uid": "lfuOmw1Vk"
-          },
-          "refId": "A"
-        }
-      ],
-      "title": "Query Engine",
-      "type": "row"
-    },
-    {
-      "collapsed": true,
-      "datasource": {
-        "type": "prometheus",
-        "uid": "lfuOmw1Vk"
-      },
-      "gridPos": {
-        "h": 1,
-        "w": 24,
-        "x": 0,
-        "y": 12
-      },
-      "id": 100,
-      "panels": [
+        },
         {
           "datasource": {
             "type": "prometheus",
@@ -5467,10 +5653,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 13
           },
-          "id": 75,
+          "id": 78,
           "options": {
             "legend": {
               "calcs": [],
@@ -5490,13 +5676,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"load_timeseries_metadata\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"load_timeseries_metadata\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"load_timeseries_metadata\"}",
               "interval": "",
               "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load timeseries metadata(avg)",
+          "title": "The time consumed of load timeseries metadata(100%)",
           "type": "timeseries"
         },
         {
@@ -5555,10 +5741,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 13
+            "x": 0,
+            "y": 21
           },
-          "id": 85,
+          "id": 86,
           "options": {
             "legend": {
               "calcs": [],
@@ -5578,13 +5764,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"load_timeseries_metadata\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"read_timeseries_metadata\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"read_timeseries_metadata\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load timeseries metadata(50%)",
+          "title": "The time consumed of read timeseries metadata(avg)",
           "type": "timeseries"
         },
         {
@@ -5643,10 +5829,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 13
+            "x": 6,
+            "y": 21
           },
-          "id": 77,
+          "id": 87,
           "options": {
             "legend": {
               "calcs": [],
@@ -5666,13 +5852,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"load_timeseries_metadata\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"read_timeseries_metadata\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load timeseries metadata(75%)",
+          "title": "The time consumed of read timeseries metadata(50%)",
           "type": "timeseries"
         },
         {
@@ -5731,10 +5917,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 13
+            "x": 12,
+            "y": 21
           },
-          "id": 78,
+          "id": 88,
           "options": {
             "legend": {
               "calcs": [],
@@ -5754,13 +5940,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"load_timeseries_metadata\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"read_timeseries_metadata\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load timeseries metadata(100%)",
+          "title": "The time consumed of read timeseries metadata(75%)",
           "type": "timeseries"
         },
         {
@@ -5819,10 +6005,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 21
           },
-          "id": 86,
+          "id": 89,
           "options": {
             "legend": {
               "calcs": [],
@@ -5842,13 +6028,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"read_timeseries_metadata\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"read_timeseries_metadata\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"read_timeseries_metadata\"}",
               "interval": "",
               "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read timeseries metadata(avg)",
+          "title": "The time consumed of read timeseries metadata(100%)",
           "type": "timeseries"
         },
         {
@@ -5907,10 +6093,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 21
+            "x": 0,
+            "y": 29
           },
-          "id": 87,
+          "id": 90,
           "options": {
             "legend": {
               "calcs": [],
@@ -5930,13 +6116,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"read_timeseries_metadata\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"timeseries_metadata_modification\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"timeseries_metadata_modification\"}[1m])",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read timeseries metadata(50%)",
+          "title": "The time consumed of timeseries metadata modification(avg)",
           "type": "timeseries"
         },
         {
@@ -5995,10 +6181,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 21
+            "x": 6,
+            "y": 29
           },
-          "id": 88,
+          "id": 91,
           "options": {
             "legend": {
               "calcs": [],
@@ -6018,13 +6204,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"read_timeseries_metadata\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"timeseries_metadata_modification\"}",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read timeseries metadata(75%)",
+          "title": "The time consumed of timeseries metadata modification(50%)",
           "type": "timeseries"
         },
         {
@@ -6083,10 +6269,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 21
+            "x": 12,
+            "y": 29
           },
-          "id": 89,
+          "id": 92,
           "options": {
             "legend": {
               "calcs": [],
@@ -6106,13 +6292,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"read_timeseries_metadata\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"timeseries_metadata_modification\"}",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read timeseries metadata(100%)",
+          "title": "The time consumed of timeseries metadata modification(75%)",
           "type": "timeseries"
         },
         {
@@ -6171,10 +6357,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 29
           },
-          "id": 90,
+          "id": 93,
           "options": {
             "legend": {
               "calcs": [],
@@ -6194,13 +6380,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"timeseries_metadata_modification\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"timeseries_metadata_modification\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"timeseries_metadata_modification\"}",
               "interval": "",
               "legendFormat": "{{type}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of timeseries metadata modification(avg)",
+          "title": "The time consumed of timeseries metadata modification(100%)",
           "type": "timeseries"
         },
         {
@@ -6259,10 +6445,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 29
+            "x": 0,
+            "y": 37
           },
-          "id": 91,
+          "id": 94,
           "options": {
             "legend": {
               "calcs": [],
@@ -6282,13 +6468,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"timeseries_metadata_modification\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"load_chunk_metadata_list\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"load_chunk_metadata_list\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of timeseries metadata modification(50%)",
+          "title": "The time consumed of load chunk metadata list(avg)",
           "type": "timeseries"
         },
         {
@@ -6347,10 +6533,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 29
+            "x": 6,
+            "y": 37
           },
-          "id": 92,
+          "id": 95,
           "options": {
             "legend": {
               "calcs": [],
@@ -6370,13 +6556,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"timeseries_metadata_modification\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"load_chunk_metadata_list\"}",
               "interval": "",
-              "legendFormat": "{{type}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of timeseries metadata modification(75%)",
+          "title": "The time consumed of load chunk metadata list(50%)",
           "type": "timeseries"
         },
         {
@@ -6435,10 +6621,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 29
+            "x": 12,
+            "y": 37
           },
-          "id": 93,
+          "id": 96,
           "options": {
             "legend": {
               "calcs": [],
@@ -6458,13 +6644,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"timeseries_metadata_modification\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"load_chunk_metadata_list\"}",
               "interval": "",
-              "legendFormat": "{{type}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of timeseries metadata modification(100%)",
+          "title": "The time consumed of load chunk metadata list(75%)",
           "type": "timeseries"
         },
         {
@@ -6523,10 +6709,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 37
           },
-          "id": 94,
+          "id": 97,
           "options": {
             "legend": {
               "calcs": [],
@@ -6546,13 +6732,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"load_chunk_metadata_list\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"load_chunk_metadata_list\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"load_chunk_metadata_list\"}",
               "interval": "",
               "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load chunk metadata list(avg)",
+          "title": "The time consumed of load chunk metadata list(100%)",
           "type": "timeseries"
         },
         {
@@ -6611,10 +6797,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 37
+            "x": 0,
+            "y": 45
           },
-          "id": 95,
+          "id": 98,
           "options": {
             "legend": {
               "calcs": [],
@@ -6634,13 +6820,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"load_chunk_metadata_list\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"chunk_metadata_modification\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"chunk_metadata_modification\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}-{{from}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load chunk metadata list(50%)",
+          "title": "The time consumed of chunk metadata modification(avg)",
           "type": "timeseries"
         },
         {
@@ -6648,6 +6834,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -6699,10 +6886,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 37
+            "x": 6,
+            "y": 45
           },
-          "id": 96,
+          "id": 101,
           "options": {
             "legend": {
               "calcs": [],
@@ -6722,13 +6909,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"load_chunk_metadata_list\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"chunk_metadata_modification\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load chunk metadata list(75%)",
+          "title": "The time consumed of chunk metadata modification(50%)",
           "type": "timeseries"
         },
         {
@@ -6736,6 +6923,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -6787,10 +6975,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 37
+            "x": 12,
+            "y": 45
           },
-          "id": 97,
+          "id": 102,
           "options": {
             "legend": {
               "calcs": [],
@@ -6810,13 +6998,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"load_chunk_metadata_list\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"chunk_metadata_modification\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of load chunk metadata list(100%)",
+          "title": "The time consumed of chunk metadata modification(75%)",
           "type": "timeseries"
         },
         {
@@ -6824,6 +7012,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -6875,10 +7064,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 45
           },
-          "id": 98,
+          "id": 103,
           "options": {
             "legend": {
               "calcs": [],
@@ -6898,13 +7087,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"chunk_metadata_modification\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"chunk_metadata_modification\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"chunk_metadata_modification\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata modification(avg)",
+          "title": "The time consumed of chunk metadata modification(100%)",
           "type": "timeseries"
         },
         {
@@ -6912,7 +7101,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -6964,10 +7152,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 45
+            "x": 0,
+            "y": 53
           },
-          "id": 101,
+          "id": 104,
           "options": {
             "legend": {
               "calcs": [],
@@ -6987,13 +7175,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"chunk_metadata_modification\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"chunk_metadata_filter\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"chunk_metadata_filter\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}-{{from}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata modification(50%)",
+          "title": "The time consumed of chunk metadata filter(avg)",
           "type": "timeseries"
         },
         {
@@ -7053,10 +7241,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 45
+            "x": 6,
+            "y": 53
           },
-          "id": 102,
+          "id": 105,
           "options": {
             "legend": {
               "calcs": [],
@@ -7076,13 +7264,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"chunk_metadata_modification\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"chunk_metadata_filter\"}",
               "interval": "",
               "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata modification(75%)",
+          "title": "The time consumed of chunk metadata filter(50%)",
           "type": "timeseries"
         },
         {
@@ -7142,10 +7330,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 45
+            "x": 12,
+            "y": 53
           },
-          "id": 103,
+          "id": 106,
           "options": {
             "legend": {
               "calcs": [],
@@ -7165,13 +7353,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"chunk_metadata_modification\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"chunk_metadata_filter\"}",
               "interval": "",
               "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata modification(100%)",
+          "title": "The time consumed of chunk metadata filter(75%)",
           "type": "timeseries"
         },
         {
@@ -7179,6 +7367,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -7230,10 +7419,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 53
           },
-          "id": 104,
+          "id": 107,
           "options": {
             "legend": {
               "calcs": [],
@@ -7253,13 +7442,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"chunk_metadata_filter\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"chunk_metadata_filter\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"chunk_metadata_filter\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata filter(avg)",
+          "title": "The time consumed of chunk metadata filter(100%)",
           "type": "timeseries"
         },
         {
@@ -7267,7 +7456,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -7319,10 +7507,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 53
+            "x": 0,
+            "y": 61
           },
-          "id": 105,
+          "id": 108,
           "options": {
             "legend": {
               "calcs": [],
@@ -7342,13 +7530,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"chunk_metadata_filter\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"construct_chunk_reader\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"construct_chunk_reader\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}-{{from}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata filter(50%)",
+          "title": "The time consumed of construct chunk reader(avg)",
           "type": "timeseries"
         },
         {
@@ -7408,10 +7596,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 53
+            "x": 6,
+            "y": 61
           },
-          "id": 106,
+          "id": 109,
           "options": {
             "legend": {
               "calcs": [],
@@ -7431,13 +7619,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"chunk_metadata_filter\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"construct_chunk_reader\"}",
               "interval": "",
               "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata filter(75%)",
+          "title": "The time consumed of construct chunk reader(50%)",
           "type": "timeseries"
         },
         {
@@ -7497,10 +7685,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 53
+            "x": 12,
+            "y": 61
           },
-          "id": 107,
+          "id": 110,
           "options": {
             "legend": {
               "calcs": [],
@@ -7520,13 +7708,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"chunk_metadata_filter\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"construct_chunk_reader\"}",
               "interval": "",
               "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of chunk metadata filter(100%)",
+          "title": "The time consumed of construct chunk reader(75%)",
           "type": "timeseries"
         },
         {
@@ -7534,6 +7722,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -7585,10 +7774,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 61
           },
-          "id": 108,
+          "id": 111,
           "options": {
             "legend": {
               "calcs": [],
@@ -7608,13 +7797,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"construct_chunk_reader\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"construct_chunk_reader\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"construct_chunk_reader\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of construct chunk reader(avg)",
+          "title": "The time consumed of construct chunk reader(100%)",
           "type": "timeseries"
         },
         {
@@ -7622,15 +7811,12 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -7674,10 +7860,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 61
+            "x": 0,
+            "y": 69
           },
-          "id": 109,
+          "id": 112,
           "options": {
             "legend": {
               "calcs": [],
@@ -7697,13 +7883,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"construct_chunk_reader\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"read_chunk\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"read_chunk\"}[1m])",
               "interval": "",
-              "legendFormat": "{{type}}-{{from}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of construct chunk reader(50%)",
+          "title": "The time consumed of read chunk(avg)",
           "type": "timeseries"
         },
         {
@@ -7718,8 +7904,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -7763,10 +7947,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 61
+            "x": 6,
+            "y": 69
           },
-          "id": 110,
+          "id": 113,
           "options": {
             "legend": {
               "calcs": [],
@@ -7786,13 +7970,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"construct_chunk_reader\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"read_chunk\"}",
               "interval": "",
-              "legendFormat": "{{type}}-{{from}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of construct chunk reader(75%)",
+          "title": "The time consumed of read chunk(50%)",
           "type": "timeseries"
         },
         {
@@ -7807,8 +7991,6 @@
                 "mode": "palette-classic"
               },
               "custom": {
-                "axisCenteredZero": false,
-                "axisColorMode": "text",
                 "axisLabel": "",
                 "axisPlacement": "auto",
                 "barAlignment": 0,
@@ -7852,10 +8034,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 61
+            "x": 12,
+            "y": 69
           },
-          "id": 111,
+          "id": 114,
           "options": {
             "legend": {
               "calcs": [],
@@ -7875,13 +8057,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"construct_chunk_reader\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"read_chunk\"}",
               "interval": "",
-              "legendFormat": "{{type}}-{{from}}",
+              "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of construct chunk reader(100%)",
+          "title": "The time consumed of read chunk(75%)",
           "type": "timeseries"
         },
         {
@@ -7889,6 +8071,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -7938,10 +8121,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 69
           },
-          "id": 112,
+          "id": 115,
           "options": {
             "legend": {
               "calcs": [],
@@ -7961,13 +8144,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"read_chunk\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"read_chunk\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"read_chunk\"}",
               "interval": "",
               "legendFormat": "{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read chunk(avg)",
+          "title": "The time consumed of read chunk(100%)",
           "type": "timeseries"
         },
         {
@@ -7975,7 +8158,6 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
-          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -8025,10 +8207,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 6,
-            "y": 69
+            "x": 0,
+            "y": 77
           },
-          "id": 113,
+          "id": 116,
           "options": {
             "legend": {
               "calcs": [],
@@ -8048,13 +8230,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"read_chunk\"}",
+              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"init_chunk_reader\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"init_chunk_reader\"}[1m])",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{type}}_{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read chunk(50%)",
+          "title": "The time consumed of init chunk reader(avg)",
           "type": "timeseries"
         },
         {
@@ -8112,10 +8294,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 12,
-            "y": 69
+            "x": 6,
+            "y": 77
           },
-          "id": 114,
+          "id": 117,
           "options": {
             "legend": {
               "calcs": [],
@@ -8135,13 +8317,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"read_chunk\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.5\", stage=\"init_chunk_reader\"}",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read chunk(75%)",
+          "title": "The time consumed of init chunk reader(50%)",
           "type": "timeseries"
         },
         {
@@ -8199,10 +8381,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 18,
-            "y": 69
+            "x": 12,
+            "y": 77
           },
-          "id": 115,
+          "id": 118,
           "options": {
             "legend": {
               "calcs": [],
@@ -8222,13 +8404,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"read_chunk\"}",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"0.75\", stage=\"init_chunk_reader\"}",
               "interval": "",
-              "legendFormat": "{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of read chunk(100%)",
+          "title": "The time consumed of init chunk reader(75%)",
           "type": "timeseries"
         },
         {
@@ -8236,6 +8418,7 @@
             "type": "prometheus",
             "uid": "${DS_PROMETHEUS}"
           },
+          "description": "",
           "fieldConfig": {
             "defaults": {
               "color": {
@@ -8285,10 +8468,10 @@
           "gridPos": {
             "h": 8,
             "w": 6,
-            "x": 0,
+            "x": 18,
             "y": 77
           },
-          "id": 116,
+          "id": 119,
           "options": {
             "legend": {
               "calcs": [],
@@ -8308,13 +8491,13 @@
                 "uid": "${DS_PROMETHEUS}"
               },
               "exemplar": true,
-              "expr": "rate(series_scan_cost_seconds_sum{instance=~\"$instance\", stage=\"init_chunk_reader\"}[1m]) / rate(series_scan_cost_seconds_count{instance=~\"$instance\", stage=\"init_chunk_reader\"}[1m])",
+              "expr": "series_scan_cost_seconds{instance=~\"${instance}\", quantile=\"1.0\", stage=\"init_chunk_reader\"}",
               "interval": "",
-              "legendFormat": "{{type}}_{{from}}",
+              "legendFormat": "{{type}}-{{from}}",
               "refId": "A"
             }
           ],
-          "title": "The time consumed of init chunk reader(avg)",
... 10102 lines suppressed ...