Sign InTry Free

Key Monitoring Metrics of TiKV

If you use TiDB Ansible to deploy the TiDB cluster, the monitoring system is deployed at the same time. For more information, see Overview of the Monitoring Framework.

The Grafana dashboard is divided into a series of sub dashboards which include Overview, PD, TiDB, TiKV, Node_exporter, Disk Performance, and so on. A lot of metrics are there to help you diagnose.

You can get an overview of the component TiKV status from the TiKV dashboard, where the key metrics are displayed. This document provides a detailed description of these key metrics.

Key metrics description

To understand the key metrics displayed on the Overview dashboard, check the following table:

ServicePanel nameDescriptionNormal range
ClusterStore sizeThe storage size of each TiKV instance
ClusterAvailable sizeThe available capacity size of each TiKV instance
ClusterCapacity sizeThe capacity size of each TiKV instance
ClusterCPUThe CPU usage of each TiKV instance
ClusterMemoryThe memory usage of each TiKV instance
ClusterIO utilizationThe I/O utilization of each TiKV instance
ClusterMBpsThe total bytes and keys of read and write on each TiKV instance
ClusterQPSThe QPS of different kinds of commands in each TiKV instance
ClusterErrpsThe total number of the gRPC message failures
ClusterLeaderThe number of leaders on each TiKV instance
ClusterRegionThe number of Regions on each TiKV instance
ErrorsServer is busyIt contains some kinds of events such as write stall, channel full, scheduler busy, and coprocessor full, which will make the TiKV instance unavailable temporarily.
ErrorsServer report failuresThe total number of reporting failure messagesIt should be 0 in normal case.
ErrorsRaftstore errorThe number of different raftstore errors on each TiKV instance
ErrorsScheduler errorThe number of different scheduler errors on each TiKV instance
ErrorsCoprocessor errorThe number of different coprocessor errors on each TiKV instance
ErrorsgRPC message errorThe number of different gRPC message errors on each TiKV instance
ErrorsLeader dropThe count of dropped leader in each TiKV instance
ErrorsLeader missingThe count of missing leader in each TiKV instance
ServerLeaderThe number of leaders on each TiKV instance
ServerRegionThe number of Regions on each TiKV instance
ServerCF sizeThe total size of each column family
ServerStore sizeThe storage size of each TiKV instance
ServerChannel fullThe total number of channel full errors on each TiKV instanceIt should be 0 in normal case.
ServerServer report failuresThe total number of reporting failure messages
ServerRegion average written keysThe average rate of writing keys to Regions on each TiKV instance
ServerRegion average written bytesThe average rate of writing bytes to Regions on each TiKV instance
ServerActive written leadersThe number of active leaders on each TiKV instance
ServerApproximate Region sizeThe approximate Region size
Raft IOApply log durationThe time consumed when Raft applies log
Raft IOApply log duration per serverThe time consumed when Raft applies log on each TiKV instance
Raft IOAppend log durationThe time consumed when Raft appends log
Raft IOAppend log duration per serverThe time consumed when Raft appends log on each TiKV instance
Raft processReady handledThe count of different ready type of Raft
Raft processProcess ready duration per serverThe time consumed when the peer processes ready in RaftIt should be less than 2s in .9999.
Raft processProcess tick duration per serverThe time consumed when the peer processes tick in Raft
Raft process0.99 Duration of raftstore eventsThe time consumed by raftstore events in .99
Raft messageSent messages per serverThe number of Raft messages sent by each TiKV instance
Raft messageFlush messages per serverThe number of Raft messages flushed by each TiKV instance
Raft messageReceive messages per serverThe number of Raft messages received by each TiKV instance
Raft messageMessagesThe number of different types of Raft messages that are sent
Raft messageVoteThe total number of vote messages that are sent in Raft
Raft messageRaft dropped messagesThe number of different types of Raft messages that are dropped
Raft proposeRaft proposals per readyThe proposal count of all Regions in a mio tick
Raft proposeRaft read/write proposalsThe total number of different kinds of proposals
Raft proposeRaft read proposals per serverThe number of read proposals which are made by each TiKV instance
Raft proposeRaft write proposals per serverThe number of write proposals which are made by each TiKV instance
Raft proposePropose wait durationThe wait time of each proposal
Raft proposePropose wait duration per serverThe wait time of each proposal in each TiKV instance
Raft proposeRaft log speedThe speed that peers propose log
Raft adminAdmin proposalsThe number of admin proposals
Raft adminAdmin applyThe number of the processed apply command
Raft adminCheck splitThe number of raftstore split check
Raft admin99.99% Check split durationThe time consumed when running split check in .9999
Local readerLocal reader requestsThe number of rejections from the local read thread and The number of total requests
Local readerLocal read requests durationThe wait time of local read requests
Local readerLocal read requests batch sizeThe batch size of local read requests
StorageStorage command totalThe total count of different kinds of commands received
StorageStorage async request errorThe total number of engine asynchronous request errors
StorageStorage async snapshot durationThe time consumed by processing asynchronous snapshot requestsIt should be less than 1s in .99.
StorageStorage async write durationThe time consumed by processing asynchronous write requestsIt should be less than 1s in .99.
SchedulerScheduler stage totalThe total number of commands on each stageThere should not be lots of errors in a short time.
SchedulerScheduler priority commandsThe count of different priority commands
SchedulerScheduler pending commandsThe count of pending commands on each TiKV instance
Scheduler - batch_getScheduler stage totalThe total number of commands on each stage in batch_get commandThere should not be lots of errors in a short time.
Scheduler - batch_getScheduler command durationThe time consumed when executing batch_get commandIt should be less than 1s.
Scheduler - batch_getScheduler latch wait durationThe time which is caused by latch wait in batch_get commandIt should be less than 1s.
Scheduler - batch_getScheduler keys readThe count of keys read by a batch_get command
Scheduler - batch_getScheduler keys writtenThe count of keys written by a batch_get command
Scheduler - batch_getScheduler scan detailsThe keys scan details of each CF when executing batch_get command
Scheduler - batch_getScheduler scan details [lock]The keys scan details of lock CF when executing batch_get command
Scheduler - batch_getScheduler scan details [write]The keys scan details of write CF when executing batch_get command
Scheduler - batch_getScheduler scan details [default]The keys scan details of default CF when executing batch_get command
Scheduler - cleanupScheduler stage totalThe total number of commands on each stage in cleanup commandThere should not be lots of errors in a short time.
Scheduler - cleanupScheduler command durationThe time consumed when executing cleanup commandIt should be less than 1s.
Scheduler - cleanupScheduler latch wait durationThe time which is caused by latch wait in cleanup commandIt should be less than 1s.
Scheduler - cleanupScheduler keys readThe count of keys read by a cleanup command
Scheduler - cleanupScheduler keys writtenThe count of keys written by a cleanup command
Scheduler - cleanupScheduler scan detailsThe keys scan details of each CF when executing cleanup command
Scheduler - cleanupScheduler scan details [lock]The keys scan details of lock CF when executing cleanup command
Scheduler - cleanupScheduler scan details [write]The keys scan details of write CF when executing cleanup command
Scheduler - cleanupScheduler scan details [default]The keys scan details of default CF when executing cleanup command
Scheduler - commitScheduler stage totalThe total number of commands on each stage in commit commandThere should not be lots of errors in a short time.
Scheduler - commitScheduler command durationThe time consumed when executing commit commandIt should be less than 1s.
Scheduler - commitScheduler latch wait durationThe time which is caused by latch wait in commit commandIt should be less than 1s.
Scheduler - commitScheduler keys readThe count of keys read by a commit command
Scheduler - commitScheduler keys writtenThe count of keys written by a commit command
Scheduler - commitScheduler scan detailsThe keys scan details of each CF when executing commit command
Scheduler - commitScheduler scan details [lock]The keys scan details of lock CF when executing commit command
Scheduler - commitScheduler scan details [write]The keys scan details of write CF when executing commit command
Scheduler - commitScheduler scan details [default]The keys scan details of default CF when executing commit command
Scheduler - gcScheduler stage totalThe total number of commands on each stage in gc commandThere should not be lots of errors in a short time.
Scheduler - gcScheduler command durationThe time consumed when executing gc commandIt should be less than 1s.
Scheduler - gcScheduler latch wait durationThe time which is caused by latch wait in gc commandIt should be less than 1s.
Scheduler - gcScheduler keys readThe count of keys read by a gc command
Scheduler - gcScheduler keys writtenThe count of keys written by a gc command
Scheduler - gcScheduler scan detailsThe keys scan details of each CF when executing gc command
Scheduler - gcScheduler scan details [lock]The keys scan details of lock CF when executing gc command
Scheduler - gcScheduler scan details [write]The keys scan details of write CF when executing gc command
Scheduler - gcScheduler scan details [default]The keys scan details of default CF when executing gc command
Scheduler - getScheduler stage totalThe total number of commands on each stage in get commandThere should not be lots of errors in a short time.
Scheduler - getScheduler command durationThe time consumed when executing get commandIt should be less than 1s.
Scheduler - getScheduler latch wait durationThe time which is caused by latch wait in get commandIt should be less than 1s.
Scheduler - getScheduler keys readThe count of keys read by a get command
Scheduler - getScheduler keys writtenThe count of keys written by a get command
Scheduler - getScheduler scan detailsThe keys scan details of each CF when executing get command
Scheduler - getScheduler scan details [lock]The keys scan details of lock CF when executing get command
Scheduler - getScheduler scan details [write]The keys scan details of write CF when executing get command
Scheduler - getScheduler scan details [default]The keys scan details of default CF when executing get command
Scheduler - key_mvccScheduler stage totalThe total number of commands on each stage in key_mvcc commandThere should not be lots of errors in a short time.
Scheduler - key_mvccScheduler command durationThe time consumed when executing key_mvcc commandIt should be less than 1s.
Scheduler - key_mvccScheduler latch wait durationThe time which is caused by latch wait in key_mvcc commandIt should be less than 1s.
Scheduler - key_mvccScheduler keys readThe count of keys read by a key_mvcc command
Scheduler - key_mvccScheduler keys writtenThe count of keys written by a key_mvcc command
Scheduler - key_mvccScheduler scan detailsThe keys scan details of each CF when executing key_mvcc command
Scheduler - key_mvccScheduler scan details [lock]The keys scan details of lock CF when executing key_mvcc command
Scheduler - key_mvccScheduler scan details [write]The keys scan details of write CF when executing key_mvcc command
Scheduler - key_mvccScheduler scan details [default]The keys scan details of default CF when executing key_mvcc command
Scheduler - prewriteScheduler stage totalThe total number of commands on each stage in prewrite commandThere should not be lots of errors in a short time.
Scheduler - prewriteScheduler command durationThe time consumed when executing prewrite commandIt should be less than 1s.
Scheduler - prewriteScheduler latch wait durationThe time which is caused by latch wait in prewrite commandIt should be less than 1s.
Scheduler - prewriteScheduler keys readThe count of keys read by a prewrite command
Scheduler - prewriteScheduler keys writtenThe count of keys written by a prewrite command
Scheduler - prewriteScheduler scan detailsThe keys scan details of each CF when executing prewrite command
Scheduler - prewriteScheduler scan details [lock]The keys scan details of lock CF when executing prewrite command
Scheduler - prewriteScheduler scan details [write]The keys scan details of write CF when executing prewrite command
Scheduler - prewriteScheduler scan details [default]The keys scan details of default CF when executing prewrite command
Scheduler - resolve_lockScheduler stage totalThe total number of commands on each stage in resolve_lock commandThere should not be lots of errors in a short time.
Scheduler - resolve_lockScheduler command durationThe time consumed when executing resolve_lock commandIt should be less than 1s.
Scheduler - resolve_lockScheduler latch wait durationThe time which is caused by latch wait in resolve_lock commandIt should be less than 1s.
Scheduler - resolve_lockScheduler keys readThe count of keys read by a resolve_lock command
Scheduler - resolve_lockScheduler keys writtenThe count of keys written by a resolve_lock command
Scheduler - resolve_lockScheduler scan detailsThe keys scan details of each CF when executing resolve_lock command
Scheduler - resolve_lockScheduler scan details [lock]The keys scan details of lock CF when executing resolve_lock command
Scheduler - resolve_lockScheduler scan details [write]The keys scan details of write CF when executing resolve_lock command
Scheduler - resolve_lockScheduler scan details [default]The keys scan details of default CF when executing resolve_lock command
Scheduler - scanScheduler stage totalThe total number of commands on each stage in scan commandThere should not be lots of errors in a short time.
Scheduler - scanScheduler command durationThe time consumed when executing scan commandIt should be less than 1s.
Scheduler - scanScheduler latch wait durationThe time which is caused by latch wait in scan commandIt should be less than 1s.
Scheduler - scanScheduler keys readThe count of keys read by a scan command
Scheduler - scanScheduler keys writtenThe count of keys written by a scan command
Scheduler - scanScheduler scan detailsThe keys scan details of each CF when executing scan command
Scheduler - scanScheduler scan details [lock]The keys scan details of lock CF when executing scan command
Scheduler - scanScheduler scan details [write]The keys scan details of write CF when executing scan command
Scheduler - scanScheduler scan details [default]The keys scan details of default CF when executing scan command
Scheduler - scan_lockScheduler stage totalThe total number of commands on each stage in scan_lock commandThere should not be lots of errors in a short time.
Scheduler - scan_lockScheduler command durationThe time consumed when executing scan_lock commandIt should be less than 1s.
Scheduler - scan_lockScheduler latch wait durationThe time which is caused by latch wait in scan_lock commandIt should be less than 1s.
Scheduler - scan_lockScheduler keys readThe count of keys read by a scan_lock command
Scheduler - scan_lockScheduler keys writtenThe count of keys written by a scan_lock command
Scheduler - scan_lockScheduler scan detailsThe keys scan details of each CF when executing scan_lock command
Scheduler - scan_lockScheduler scan details [lock]The keys scan details of lock CF when executing scan_lock command
Scheduler - scan_lockScheduler scan details [write]The keys scan details of write CF when executing scan_lock command
Scheduler - scan_lockScheduler scan details [default]The keys scan details of default CF when executing scan_lock command
Scheduler - start_ts_mvccScheduler stage totalThe total number of commands on each stage in start_ts_mvcc commandThere should not be lots of errors in a short time.
Scheduler - start_ts_mvccScheduler command durationThe time consumed when executing start_ts_mvcc commandIt should be less than 1s.
Scheduler - start_ts_mvccScheduler latch wait durationThe time which is caused by latch wait in start_ts_mvcc commandIt should be less than 1s.
Scheduler - start_ts_mvccScheduler keys readThe count of keys read by a start_ts_mvcc command
Scheduler - start_ts_mvccScheduler keys writtenThe count of keys written by a start_ts_mvcc command
Scheduler - start_ts_mvccScheduler scan detailsThe keys scan details of each CF when executing start_ts_mvcc command
Scheduler - start_ts_mvccScheduler scan details [lock]The keys scan details of lock CF when executing start_ts_mvcc command
Scheduler - start_ts_mvccScheduler scan details [write]The keys scan details of write CF when executing start_ts_mvcc command
Scheduler - start_ts_mvccScheduler scan details [default]The keys scan details of default CF when executing start_ts_mvcc command
Scheduler - unsafe_destroy_rangeScheduler stage totalThe total number of commands on each stage in unsafe_destroy_range commandThere should not be lots of errors in a short time.
Scheduler - unsafe_destroy_rangeScheduler command durationThe time consumed when executing unsafe_destroy_range commandIt should be less than 1s.
Scheduler - unsafe_destroy_rangeScheduler latch wait durationThe time which is caused by latch wait in unsafe_destroy_range commandIt should be less than 1s.
Scheduler - unsafe_destroy_rangeScheduler keys readThe count of keys read by a unsafe_destroy_range command
Scheduler - unsafe_destroy_rangeScheduler keys writtenThe count of keys written by a unsafe_destroy_range command
Scheduler - unsafe_destroy_rangeScheduler scan detailsThe keys scan details of each CF when executing unsafe_destroy_range command
Scheduler - unsafe_destroy_rangeScheduler scan details [lock]The keys scan details of lock CF when executing unsafe_destroy_range command
Scheduler - unsafe_destroy_rangeScheduler scan details [write]The keys scan details of write CF when executing unsafe_destroy_range command
Scheduler - unsafe_destroy_rangeScheduler scan details [default]The keys scan details of default CF when executing unsafe_destroy_range command
CoprocessorRequest durationThe time consumed when handling coprocessor read requests
CoprocessorWait durationThe time consumed when coprocessor requests are wait for being handledIt should be less than 10s in .9999.
CoprocessorHandle durationThe time consumed when handling coprocessor requests
Coprocessor95% Request duration by storeThe time consumed when handling coprocessor read requests in each TiKV instance
Coprocessor95% Wait duration by storeThe time consumed when coprocessor requests are wait for being handled in each TiKV instance
Coprocessor95% Handle duration by storeThe time consumed when handling coprocessor requests in each TiKV instance
CoprocessorRequest errorsThe total number of the push down request errorsThere should not be lots of errors in a short time.
CoprocessorDAG executorsThe total number of DAG executors
CoprocessorScan keysThe number of keys that each request scans
CoprocessorScan detailsThe scan details for each CF
CoprocessorTable Scan - Details by CFThe table scan details for each CF
CoprocessorIndex Scan - Details by CFThe index scan details for each CF
CoprocessorTable Scan - Perf StatisticsThe total number of RocksDB internal operations from PerfContext when executing table scan
CoprocessorIndex Scan - Perf StatisticsThe total number of RocksDB internal operations from PerfContext when executing index scan
GCMVCC versionsThe number of versions for each key
GCMVCC delete versionsThe number of versions deleted by GC for each key
GCGC tasksThe count of GC tasks processed by gc_worker
GCGC tasks DurationThe time consumed when executing GC tasks
GCGC keys (write CF)The count of keys in write CF affected during GC
GCTiDB GC actions resultThe TiDB GC action result on Region level
GCTiDB GC worker actionsThe count of TiDB GC worker actions
GCTiDB GC secondsThe time consumed when TiDB is doing GC
GCTiDB GC failureThe count of TiDB GC job failure
GCGC lifetimeThe lifetime of TiDB GC
GCGC intervalThe interval of TiDB GC
SnapshotRate snapshot messageThe rate of Raft snapshot messages sent
Snapshot99% Handle snapshot durationThe time consumed when handling snapshots
SnapshotSnapshot state countThe number of snapshots in different states
Snapshot99.99% Snapshot sizeThe snapshot size in .9999
Snapshot99.99% Snapshot KV countThe number of KV within a snapshot in .9999
TaskWorker handled tasksThe number of tasks handled by worker
TaskWorker pending tasksCurrent pending and running tasks of workerIt should be less than 1000.
TaskFuturePool pending tasksCurrent pending and running tasks of future_pool
Thread CPURaft store CPUThe CPU utilization of raftstore threadThe CPU usage should be less than 80%.
Thread CPUAsync apply CPUThe CPU utilization of async applyThe CPU usage should be less than 90%.
Thread CPUScheduler CPUThe CPU utilization of schedulerThe CPU usage should be less than 80%.
Thread CPUScheduler Worker CPUThe CPU utilization of scheduler worker
Thread CPUStorage ReadPool CPUThe CPU utilization of readpool
Thread CPUCoprocessor CPUThe CPU utilization of coprocessor
Thread CPUSnapshot worker CPUThe CPU utilization of snapshot worker
Thread CPUSplit check CPUThe CPU utilization of split check
Thread CPURocksDB CPUThe CPU utilization of RocksDB
Thread CPUgRPC poll CPUThe CPU utilization of gRPCThe CPU usage should be less than 80%.
RocksDB - kvGet operationsThe count of get operations
RocksDB - kvGet durationThe time consumed when executing get operation
RocksDB - kvSeek operationsThe count of seek operations
RocksDB - kvSeek durationThe time consumed when executing seek operation
RocksDB - kvWrite operationsThe count of write operations
RocksDB - kvWrite durationThe time consumed when executing write operation
RocksDB - kvWAL sync operationsThe count of WAL sync operations
RocksDB - kvWAL sync durationThe time consumed when executing WAL sync operation
RocksDB - kvCompaction operationsThe count of compaction and flush operations
RocksDB - kvCompaction durationThe time consumed when executing compaction and flush operation
RocksDB - kvSST read durationThe time consumed when reading SST files
RocksDB - kvWrite stall durationThe time which is caused by write stallIt should be 0 in normal case.
RocksDB - kvMemtable sizeThe memtable size of each column family
RocksDB - kvMemtable hitThe hit rate of memtable
RocksDB - kvBlock cache sizeThe block cache size of each column family
RocksDB - kvBlock cache hitThe hit rate of block cache
RocksDB - kvBlock cache flowThe flow of different kinds of block cache operations
RocksDB - kvBlock cache operationsThe count of different kinds of block cache operations
RocksDB - kvKeys flowThe flow of different kinds of operations on keys
RocksDB - kvTotal keysThe count of keys in each column family
RocksDB - kvRead flowThe flow of different kinds of read operations
RocksDB - kvBytes / ReadThe bytes per read
RocksDB - kvWrite flowThe flow of different kinds of write operations
RocksDB - kvBytes / WriteThe bytes per write
RocksDB - kvCompaction flowThe flow of different kinds of compaction operations
RocksDB - kvCompaction pending bytesThe pending bytes when executing compaction
RocksDB - kvRead amplificationThe read amplification in each TiKV instance
RocksDB - kvCompression ratioThe compression ratio of each level
RocksDB - kvNumber of snapshotsThe number of snapshot of each TiKV instance
RocksDB - kvOldest snapshots durationThe time that the oldest unreleased snapshot survivals
RocksDB - kvNumber files at each levelThe number of SST files for different column families in each level
RocksDB - kvIngest SST duration secondsThe time consumed when ingesting SST files
RocksDB - kvStall conditions changed of each CFStall conditions changed of each column family
RocksDB - raftGet operationsThe count of get operations
RocksDB - raftGet durationThe time consumed when executing get operation
RocksDB - raftSeek operationsThe count of seek operations
RocksDB - raftSeek durationThe time consumed when executing seek operation
RocksDB - raftWrite operationsThe count of write operations
RocksDB - raftWrite durationThe time consumed when executing write operation
RocksDB - raftWAL sync operationsThe count of WAL sync operations
RocksDB - raftWAL sync durationThe time consumed when executing WAL sync operation
RocksDB - raftCompaction operationsThe count of compaction and flush operations
RocksDB - raftCompaction durationThe time consumed when executing compaction and flush operation
RocksDB - raftSST read durationThe time consumed when reading SST files
RocksDB - raftWrite stall durationThe time which is caused by write stallIt should be 0 in normal case.
RocksDB - raftMemtable sizeThe memtable size of each column family
RocksDB - raftMemtable hitThe hit rate of memtable
RocksDB - raftBlock cache sizeThe block cache size of each column family
RocksDB - raftBlock cache hitThe hit rate of block cache
RocksDB - raftBlock cache flowThe flow of different kinds of block cache operations
RocksDB - raftBlock cache operationsThe count of different kinds of block cache operations
RocksDB - raftKeys flowThe flow of different kinds of operations on keys
RocksDB - raftTotal keysThe count of keys in each column family
RocksDB - raftRead flowThe flow of different kinds of read operations
RocksDB - raftBytes / ReadThe bytes per read
RocksDB - raftWrite flowThe flow of different kinds of write operations
RocksDB - raftBytes / WriteThe bytes per write
RocksDB - raftCompaction flowThe flow of different kinds of compaction operations
RocksDB - raftCompaction pending BytesThe pending bytes when executing compaction
RocksDB - raftRead amplificationThe read amplification in each TiKV instance
RocksDB - raftCompression ratioThe compression ratio of each level
RocksDB - raftNumber of snapshotsThe number of snapshot of each TiKV instance
RocksDB - raftOldest snapshots durationThe time that the oldest unreleased snapshot survivals
RocksDB - raftNumber files at each levelThe number of SST files for different column families in each level
RocksDB - raftIngest SST duration secondsThe time consumed when ingesting SST files
RocksDB - raftStall conditions changed of each CFStall conditions changed of each column family
gRPCgRPC message countThe count of different kinds of gRPC message
gRPCgRPC message failedThe count of different kinds of gRPC message which is failed
gRPC99% gRPC message durationThe execution time of gRPC message
gRPCgRPC GC message countThe count of gRPC GC message
gRPC99% gRPC KV GC message durationThe execution time of gRPC GC message
PDPD requestsThe count of requests that TiKV sends to PD
PDPD request duration (average)The time consumed by requests that TiKV sends to PD
PDPD heartbeatsThe total number of PD heartbeat messages
PDPD validate peersThe total number of peers validated by the PD worker

TiKV dashboard interface

This section shows images of the service panels on the TiKV dashboard.

Cluster

TiKV Dashboard - Cluster metrics

Errors

TiKV Dashboard - Errors metrics

Server

TiKV Dashboard - Server metrics

Raft IO

TiKV Dashboard - Raft IO metrics

Raft process

TiKV Dashboard - Raft process metrics

Raft message

TiKV Dashboard - Raft message metrics

Raft propose

TiKV Dashboard - Raft propose metrics

Raft admin

TiKV Dashboard - Raft admin metrics

Local reader

TiKV Dashboard - Local reader metrics

Storage

TiKV Dashboard - Storage metrics

Scheduler

TiKV Dashboard - Scheduler metrics

Scheduler - batch_get

TiKV Dashboard - Scheduler - batch_get metrics

Scheduler - cleanup

TiKV Dashboard - Scheduler - cleanup metrics

Scheduler - commit

TiKV Dashboard - Scheduler commit metrics

Download PDF
Playground
New
One-stop & interactive experience of TiDB's capabilities WITHOUT registration.
Products
TiDB
TiDB Dedicated
TiDB Serverless
Pricing
Get Demo
Get Started
© 2024 PingCAP. All Rights Reserved.
Privacy Policy.