Apache Kudu Metrics Reference

This document applies to Apache Kudu version 1.15.0. Please consult the documentation of the appropriate release that’s applicable to the version of the Kudu cluster.

kudu-master Metrics

Warning Metrics

Metrics tagged as 'warn' are metrics which can often indicate operational oddities that may need more investigation.

cluster_replica_skew

Cluster Replica Skew   The difference between the number of replicas on the tablet server hosting the most replicas and the number of replicas on the tablet server hosting the least replicas.

Entity Type

server

Unit

tablets

Type

gauge

Level

warn

 

data_dirs_failed

Data Directories Failed   Number of data directories whose disks are currently in a failed state

Entity Type

server

Unit

data directories

Type

gauge

Level

warn

 

data_dirs_full

Data Directories Full   Number of data directories whose disks are currently full

Entity Type

server

Unit

data directories

Type

gauge

Level

warn

 

failed_elections_since_stable_leader

Failed Elections Since Stable Leader   Number of failed elections on this node since there was a stable leader. This number increments on each failed election and resets on each successful one.

Entity Type

tablet

Unit

units

Type

gauge

Level

warn

 

follower_memory_pressure_rejections

Follower Memory Pressure Rejections   Number of RPC requests rejected due to memory pressure while FOLLOWER.

Entity Type

tablet

Unit

requests

Type

counter

Level

warn

 

glog_error_messages

ERROR-level Log Messages   Number of ERROR-level log messages emitted by the application.

Entity Type

server

Unit

messages

Type

counter

Level

warn

 

glog_warning_messages

WARNING-level Log Messages   Number of WARNING-level log messages emitted by the application.

Entity Type

server

Unit

messages

Type

counter

Level

warn

 

hybrid_clock_extrapolating

Hybrid Clock Is Being Extrapolated   Whether HybridClock timestamps are extrapolated because of inability to read the underlying clock

Entity Type

server

Unit

state

Type

gauge

Level

warn

 

hybrid_clock_extrapolation_intervals

Intervals of Hybrid Clock Extrapolation   The statistics on the duration of intervals when the underlying clock was extrapolated instead of using the direct readings

Entity Type

server

Unit

seconds

Type

histogram

Level

warn

 

leader_memory_pressure_rejections

Leader Memory Pressure Rejections   Number of RPC requests rejected due to memory pressure while LEADER.

Entity Type

tablet

Unit

requests

Type

counter

Level

warn

 

maintenance_op_prepare_failed

Number Of Operations With Failed Prepare()   Number of times when calling Prepare() on a maintenance operation failed

Entity Type

server

Unit

operations

Type

counter

Level

warn

 

op_apply_queue_length

Operation Apply Queue Length   Number of operations waiting to be applied to the tablet. High queue lengths indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

server

Unit

tasks

Type

histogram

Level

warn

 

op_apply_queue_overload_rejections

Number of Rejected Write Requests Due to Queue Overloaded Error   Number of rejected write requests due to overloaded op apply queue

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

op_apply_queue_time

Operation Apply Queue Time   Time that operations spent waiting in the apply queue before being processed. High queue times indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

server

Unit

microseconds

Type

histogram

Level

warn

 

op_apply_run_time

Operation Apply Run Time   Time that operations spent being applied to the tablet. High values may indicate that the server is under-provisioned or that operations consist of very large batches.

Entity Type

server

Unit

microseconds

Type

histogram

Level

warn

 

ops_behind_leader

Operations Behind Leader   Number of operations this server believes it is behind the leader.

Entity Type

tablet

Unit

operations

Type

gauge

Level

warn

 

rpcs_queue_overflow

RPC Queue Overflows   Number of RPCs dropped because the service queue was full.

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

rpcs_timed_out_in_queue

RPC Queue Timeouts   Number of RPCs whose timeout elapsed while waiting in the service queue, and thus were not processed.

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

scanners_expired

Scanners Expired   Number of scanners that have expired due to inactivity since service start

Entity Type

server

Unit

scanners

Type

counter

Level

warn

 

spinlock_contention_time

Spinlock Contention Time   Amount of time consumed by contention on internal spinlocks since the server started. If this increases rapidly, it may indicate a performance issue in Kudu internals triggered by a particular workload and warrant investigation.

Entity Type

server

Unit

microseconds

Type

counter

Level

warn

 

sys_catalog_oversized_write_requests

System Catalog Oversized Write Requests   Number of oversized write requests to the system catalog tablet rejected since start

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

tablets_num_failed

Number of Failed Tablets   Number of failed tablets

Entity Type

server

Unit

tablets

Type

gauge

Level

warn

 

transaction_memory_limit_rejections

Tablet Op Memory Limit Rejections   Number of ops rejected because the tablet’s op memory limit was reached.

Entity Type

tablet

Unit

transactions

Type

counter

Level

warn

 

transaction_memory_pressure_rejections

Op Memory Pressure Rejections   Number of ops rejected because the tablet’s opmemory usage exceeds the op memory limit or the limitof an ancestral tracker.

Entity Type

tablet

Unit

transactions

Type

counter

Level

warn

 

Info Metrics

Metrics tagged as 'info' are generally useful metrics that operators always want to have available but may not be monitored under normal circumstances.

active_scanners

Active Scanners   Number of scanners that are currently active

Entity Type

server

Unit

scanners

Type

gauge

Level

info

 

average_diskrowset_height

Average DiskRowSet Height   Average height of the diskrowsets in this tablet replica. The larger the average height, the more uncompacted the tablet replica is.

Entity Type

tablet

Unit

units

Type

gauge

Level

info

 

block_cache_usage

Block Cache Memory Usage   Memory consumed by the block cache

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

block_manager_blocks_open_reading

Data Blocks Open For Read   Number of data blocks currently open for reading

Entity Type

server

Unit

blocks

Type

gauge

Level

info

 

block_manager_blocks_open_writing

Data Blocks Open For Write   Number of data blocks currently open for writing

Entity Type

server

Unit

blocks

Type

gauge

Level

info

 

builtin_ntp_error

Built-in NTP Latest Maximum Time Error   Latest maximum time error as tracked by built-in NTP client

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

builtin_ntp_local_clock_delta

Local Clock vs Built-In NTP True Time Delta   Delta between local clock and true time tracked by built-in NTP client; set to 2^63-1 when true time is not tracked

Entity Type

server

Unit

milliseconds

Type

gauge

Level

info

 

compact_rs_duration

RowSet Compaction Duration   Time spent compacting RowSets.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

cpu_stime

System CPU Time   Total system CPU time of the process

Entity Type

server

Unit

milliseconds

Type

counter

Level

info

 

cpu_utime

User CPU Time   Total user CPU time of the process

Entity Type

server

Unit

milliseconds

Type

counter

Level

info

 

deleted_rowset_gc_duration

Deleted Rowset GC Duration   Time spent running the maintenance operation to GC deleted rowsets.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

delta_file_lookups_per_op

Delta File Lookups per Operation   Tracks the number of delta file lookups performed by each operation. A single operation may perform several delta file lookups if the tablet is not fully compacted. High frequency of high values may indicate that compaction is falling behind.

Entity Type

tablet

Unit

probes

Type

histogram

Level

info

 

delta_major_compact_rs_duration

Major Delta Compaction Duration   Seconds spent major delta compacting.

Entity Type

tablet

Unit

seconds

Type

histogram

Level

info

 

delta_minor_compact_rs_duration

Minor Delta Compaction Duration   Time spent minor delta compacting.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

file_cache_usage

File Cache Usage   Number of entries in the file cache

Entity Type

server

Unit

entries

Type

gauge

Level

info

 

flush_dms_duration

DeltaMemStore Flush Duration   Time spent flushing DeltaMemStores.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

flush_mrs_duration

MemRowSet Flush Duration   Time spent flushing MemRowSets.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

generic_current_allocated_bytes

Heap Memory Usage   Number of bytes used by the application. This will not typically match the memory use reported by the OS, because it does not include TCMalloc overhead or memory fragmentation.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

generic_heap_size

Reserved Heap Memory   Bytes of system memory reserved by TCMalloc.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

handler_latency_kudu_consensus_ConsensusService_BulkChangeConfig

kudu.consensus.ConsensusService.BulkChangeConfig RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.BulkChangeConfig RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_ChangeConfig

kudu.consensus.ConsensusService.ChangeConfig RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.ChangeConfig RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_GetConsensusState

kudu.consensus.ConsensusService.GetConsensusState RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.GetConsensusState RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_GetLastOpId

kudu.consensus.ConsensusService.GetLastOpId RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.GetLastOpId RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_GetNodeInstance

kudu.consensus.ConsensusService.GetNodeInstance RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.GetNodeInstance RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_LeaderStepDown

kudu.consensus.ConsensusService.LeaderStepDown RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.LeaderStepDown RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_RequestConsensusVote

kudu.consensus.ConsensusService.RequestConsensusVote RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.RequestConsensusVote RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_RunLeaderElection

kudu.consensus.ConsensusService.RunLeaderElection RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.RunLeaderElection RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_StartTabletCopy

kudu.consensus.ConsensusService.StartTabletCopy RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.StartTabletCopy RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_UnsafeChangeConfig

kudu.consensus.ConsensusService.UnsafeChangeConfig RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.UnsafeChangeConfig RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_UpdateConsensus

kudu.consensus.ConsensusService.UpdateConsensus RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.UpdateConsensus RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_AddMaster

kudu.master.MasterService.AddMaster RPC Time   Microseconds spent handling kudu.master.MasterService.AddMaster RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_AlterTable

kudu.master.MasterService.AlterTable RPC Time   Microseconds spent handling kudu.master.MasterService.AlterTable RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_ChangeTServerState

kudu.master.MasterService.ChangeTServerState RPC Time   Microseconds spent handling kudu.master.MasterService.ChangeTServerState RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_ConnectToMaster

kudu.master.MasterService.ConnectToMaster RPC Time   Microseconds spent handling kudu.master.MasterService.ConnectToMaster RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_CreateTable

kudu.master.MasterService.CreateTable RPC Time   Microseconds spent handling kudu.master.MasterService.CreateTable RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_DeleteTable

kudu.master.MasterService.DeleteTable RPC Time   Microseconds spent handling kudu.master.MasterService.DeleteTable RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_GetMasterRegistration

kudu.master.MasterService.GetMasterRegistration RPC Time   Microseconds spent handling kudu.master.MasterService.GetMasterRegistration RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_GetTableLocations

kudu.master.MasterService.GetTableLocations RPC Time   Microseconds spent handling kudu.master.MasterService.GetTableLocations RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_GetTableSchema

kudu.master.MasterService.GetTableSchema RPC Time   Microseconds spent handling kudu.master.MasterService.GetTableSchema RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_GetTableStatistics

kudu.master.MasterService.GetTableStatistics RPC Time   Microseconds spent handling kudu.master.MasterService.GetTableStatistics RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_GetTabletLocations

kudu.master.MasterService.GetTabletLocations RPC Time   Microseconds spent handling kudu.master.MasterService.GetTabletLocations RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_IsAlterTableDone

kudu.master.MasterService.IsAlterTableDone RPC Time   Microseconds spent handling kudu.master.MasterService.IsAlterTableDone RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_IsCreateTableDone

kudu.master.MasterService.IsCreateTableDone RPC Time   Microseconds spent handling kudu.master.MasterService.IsCreateTableDone RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_ListMasters

kudu.master.MasterService.ListMasters RPC Time   Microseconds spent handling kudu.master.MasterService.ListMasters RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_ListTables

kudu.master.MasterService.ListTables RPC Time   Microseconds spent handling kudu.master.MasterService.ListTables RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_ListTabletServers

kudu.master.MasterService.ListTabletServers RPC Time   Microseconds spent handling kudu.master.MasterService.ListTabletServers RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_Ping

kudu.master.MasterService.Ping RPC Time   Microseconds spent handling kudu.master.MasterService.Ping RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_RefreshAuthzCache

kudu.master.MasterService.RefreshAuthzCache RPC Time   Microseconds spent handling kudu.master.MasterService.RefreshAuthzCache RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_RemoveMaster

kudu.master.MasterService.RemoveMaster RPC Time   Microseconds spent handling kudu.master.MasterService.RemoveMaster RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_ReplaceTablet

kudu.master.MasterService.ReplaceTablet RPC Time   Microseconds spent handling kudu.master.MasterService.ReplaceTablet RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_master_MasterService_TSHeartbeat

kudu.master.MasterService.TSHeartbeat RPC Time   Microseconds spent handling kudu.master.MasterService.TSHeartbeat RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_CheckLeaks

kudu.server.GenericService.CheckLeaks RPC Time   Microseconds spent handling kudu.server.GenericService.CheckLeaks RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_DumpMemTrackers

kudu.server.GenericService.DumpMemTrackers RPC Time   Microseconds spent handling kudu.server.GenericService.DumpMemTrackers RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_FlushCoverage

kudu.server.GenericService.FlushCoverage RPC Time   Microseconds spent handling kudu.server.GenericService.FlushCoverage RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_GetFlags

kudu.server.GenericService.GetFlags RPC Time   Microseconds spent handling kudu.server.GenericService.GetFlags RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_GetStatus

kudu.server.GenericService.GetStatus RPC Time   Microseconds spent handling kudu.server.GenericService.GetStatus RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_ServerClock

kudu.server.GenericService.ServerClock RPC Time   Microseconds spent handling kudu.server.GenericService.ServerClock RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_SetFlag

kudu.server.GenericService.SetFlag RPC Time   Microseconds spent handling kudu.server.GenericService.SetFlag RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_SetServerWallClockForTests

kudu.server.GenericService.SetServerWallClockForTests RPC Time   Microseconds spent handling kudu.server.GenericService.SetServerWallClockForTests RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_transactions_TxnManagerService_AbortTransaction

kudu.transactions.TxnManagerService.AbortTransaction RPC Time   Microseconds spent handling kudu.transactions.TxnManagerService.AbortTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_transactions_TxnManagerService_BeginTransaction

kudu.transactions.TxnManagerService.BeginTransaction RPC Time   Microseconds spent handling kudu.transactions.TxnManagerService.BeginTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_transactions_TxnManagerService_CommitTransaction

kudu.transactions.TxnManagerService.CommitTransaction RPC Time   Microseconds spent handling kudu.transactions.TxnManagerService.CommitTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_transactions_TxnManagerService_GetTransactionState

kudu.transactions.TxnManagerService.GetTransactionState RPC Time   Microseconds spent handling kudu.transactions.TxnManagerService.GetTransactionState RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_transactions_TxnManagerService_KeepTransactionAlive

kudu.transactions.TxnManagerService.KeepTransactionAlive RPC Time   Microseconds spent handling kudu.transactions.TxnManagerService.KeepTransactionAlive RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_BeginTabletCopySession

kudu.tserver.TabletCopyService.BeginTabletCopySession RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.BeginTabletCopySession RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_CheckSessionActive

kudu.tserver.TabletCopyService.CheckSessionActive RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.CheckSessionActive RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_EndTabletCopySession

kudu.tserver.TabletCopyService.EndTabletCopySession RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.EndTabletCopySession RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_FetchData

kudu.tserver.TabletCopyService.FetchData RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.FetchData RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_AlterSchema

kudu.tserver.TabletServerAdminService.AlterSchema RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.AlterSchema RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_CoordinateTransaction

kudu.tserver.TabletServerAdminService.CoordinateTransaction RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.CoordinateTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_CreateTablet

kudu.tserver.TabletServerAdminService.CreateTablet RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.CreateTablet RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet

kudu.tserver.TabletServerAdminService.DeleteTablet RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.DeleteTablet RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_ParticipateInTransaction

kudu.tserver.TabletServerAdminService.ParticipateInTransaction RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.ParticipateInTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_Quiesce

kudu.tserver.TabletServerAdminService.Quiesce RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.Quiesce RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Checksum

kudu.tserver.TabletServerService.Checksum RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Checksum RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_ListTablets

kudu.tserver.TabletServerService.ListTablets RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.ListTablets RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Ping

kudu.tserver.TabletServerService.Ping RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Ping RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Scan

kudu.tserver.TabletServerService.Scan RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Scan RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_ScannerKeepAlive

kudu.tserver.TabletServerService.ScannerKeepAlive RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.ScannerKeepAlive RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_SplitKeyRange

kudu.tserver.TabletServerService.SplitKeyRange RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.SplitKeyRange RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Write

kudu.tserver.TabletServerService.Write RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Write RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

hybrid_clock_error

Hybrid Clock Error   Server clock maximum error; returns 2^64-1 when unable to read the underlying clock

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

hybrid_clock_timestamp

Hybrid Clock Timestamp   Hybrid clock timestamp; returns 2^64-1 when unable to read the underlying clock

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

involuntary_context_switches

Involuntary Context Switches   Total involuntary context switches

Entity Type

server

Unit

context switches

Type

counter

Level

info

 

live_row_count

Table Live Row count   Pre-replication aggregated number of live rows in this table. Only accurate if all tablets in the table support live row counting.

Entity Type

table

Unit

rows

Type

gauge

Level

info

 

live_row_count

Tablet Live Row Count   Number of live rows in this tablet, excludes deleted rows.

Entity Type

tablet

Unit

rows

Type

gauge

Level

info

 

log_append_latency

Log Append Latency   Microseconds spent on appending to the log segment file

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

log_block_manager_blocks_under_management

Blocks Under Management   Number of data blocks currently under management

Entity Type

server

Unit

blocks

Type

gauge

Level

info

 

log_block_manager_bytes_under_management

Bytes Under Management   Number of bytes of data blocks currently under management

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

log_block_manager_containers

Number of Block Containers   Number of log block containers

Entity Type

server

Unit

log block containers

Type

gauge

Level

info

 

log_block_manager_full_containers

Number of Full Block Containers   Number of full log block containers

Entity Type

server

Unit

log block containers

Type

gauge

Level

info

 

log_gc_duration

Log GC Duration   Time spent garbage collecting the logs.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

log_gc_running

Log GCs Running   Number of log GC operations currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

info

 

log_reader_read_batch_latency

Log Read Latency   Microseconds spent reading log entry batches

Entity Type

tablet

Unit

bytes

Type

histogram

Level

info

 

log_sync_latency

Log Sync Latency   Microseconds spent on synchronizing the log segment file

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

logical_clock_timestamp

Logical Clock Timestamp   Logical clock timestamp.

Entity Type

server

Unit

units

Type

gauge

Level

info

 

maintenance_op_find_best_candidate_duration

Time Taken To Find Best Maintenance Operation   Time spent choosing a maintenance operation with highest scores among available candidates

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

memrowset_size

MemRowSet Memory Usage   Size of this tablet’s memrowset

Entity Type

tablet

Unit

bytes

Type

gauge

Level

info

 

merged_entities_count_of_server

Entities Count Merged From   Count of entities merged together when entities are merged by common attribute value.

Entity Type

server

Unit

entries

Type

gauge

Level

info

 

merged_entities_count_of_table

Entities Count Merged From   Count of entities merged together when entities are merged by common attribute value.

Entity Type

table

Unit

entries

Type

gauge

Level

info

 

merged_entities_count_of_tablet

Entities Count Merged From   Count of entities merged together when entities are merged by common attribute value.

Entity Type

tablet

Unit

entries

Type

gauge

Level

info

 

num_raft_leaders

Number of Raft Leaders   Number of tablet replicas that are Raft leaders

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

num_rowsets_on_disk

Tablet Number of Rowsets on Disk   Number of diskrowsets in this tablet

Entity Type

tablet

Unit

units

Type

gauge

Level

info

 

on_disk_data_size

Tablet Data Size On Disk   Space used by this tablet’s data blocks.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

info

 

on_disk_size

Table Size On Disk   Pre-replication aggregated disk space used by all tablets in this table, including metadata.

Entity Type

table

Unit

bytes

Type

gauge

Level

info

 

on_disk_size

Tablet Size On Disk   Space used by this tablet on disk, including metadata.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

info

 

op_prepare_queue_length

Operation Prepare Queue Length   Number of operations waiting to be prepared within this tablet. High queue lengths indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

tablet

Unit

tasks

Type

histogram

Level

info

 

op_prepare_queue_time

Operation Prepare Queue Time   Time that operations spent waiting in the prepare queue before being processed. High queue times indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

op_prepare_run_time

Operation Prepare Run Time   Time that operations spent being prepared in the tablet. High values may indicate that the server is under-provisioned or that operations are experiencing high contention with one another for locks.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_BulkChangeConfig

kudu.consensus.ConsensusService.BulkChangeConfig RPC Rejections   Number of rejected kudu.consensus.ConsensusService.BulkChangeConfig requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_ChangeConfig

kudu.consensus.ConsensusService.ChangeConfig RPC Rejections   Number of rejected kudu.consensus.ConsensusService.ChangeConfig requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_GetConsensusState

kudu.consensus.ConsensusService.GetConsensusState RPC Rejections   Number of rejected kudu.consensus.ConsensusService.GetConsensusState requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_GetLastOpId

kudu.consensus.ConsensusService.GetLastOpId RPC Rejections   Number of rejected kudu.consensus.ConsensusService.GetLastOpId requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_GetNodeInstance

kudu.consensus.ConsensusService.GetNodeInstance RPC Rejections   Number of rejected kudu.consensus.ConsensusService.GetNodeInstance requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_LeaderStepDown

kudu.consensus.ConsensusService.LeaderStepDown RPC Rejections   Number of rejected kudu.consensus.ConsensusService.LeaderStepDown requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_RequestConsensusVote

kudu.consensus.ConsensusService.RequestConsensusVote RPC Rejections   Number of rejected kudu.consensus.ConsensusService.RequestConsensusVote requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_RunLeaderElection

kudu.consensus.ConsensusService.RunLeaderElection RPC Rejections   Number of rejected kudu.consensus.ConsensusService.RunLeaderElection requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_StartTabletCopy

kudu.consensus.ConsensusService.StartTabletCopy RPC Rejections   Number of rejected kudu.consensus.ConsensusService.StartTabletCopy requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_UnsafeChangeConfig

kudu.consensus.ConsensusService.UnsafeChangeConfig RPC Rejections   Number of rejected kudu.consensus.ConsensusService.UnsafeChangeConfig requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_UpdateConsensus

kudu.consensus.ConsensusService.UpdateConsensus RPC Rejections   Number of rejected kudu.consensus.ConsensusService.UpdateConsensus requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_AddMaster

kudu.master.MasterService.AddMaster RPC Rejections   Number of rejected kudu.master.MasterService.AddMaster requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_AlterTable

kudu.master.MasterService.AlterTable RPC Rejections   Number of rejected kudu.master.MasterService.AlterTable requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_ChangeTServerState

kudu.master.MasterService.ChangeTServerState RPC Rejections   Number of rejected kudu.master.MasterService.ChangeTServerState requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_ConnectToMaster

kudu.master.MasterService.ConnectToMaster RPC Rejections   Number of rejected kudu.master.MasterService.ConnectToMaster requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_CreateTable

kudu.master.MasterService.CreateTable RPC Rejections   Number of rejected kudu.master.MasterService.CreateTable requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_DeleteTable

kudu.master.MasterService.DeleteTable RPC Rejections   Number of rejected kudu.master.MasterService.DeleteTable requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_GetMasterRegistration

kudu.master.MasterService.GetMasterRegistration RPC Rejections   Number of rejected kudu.master.MasterService.GetMasterRegistration requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_GetTableLocations

kudu.master.MasterService.GetTableLocations RPC Rejections   Number of rejected kudu.master.MasterService.GetTableLocations requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_GetTableSchema

kudu.master.MasterService.GetTableSchema RPC Rejections   Number of rejected kudu.master.MasterService.GetTableSchema requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_GetTableStatistics

kudu.master.MasterService.GetTableStatistics RPC Rejections   Number of rejected kudu.master.MasterService.GetTableStatistics requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_GetTabletLocations

kudu.master.MasterService.GetTabletLocations RPC Rejections   Number of rejected kudu.master.MasterService.GetTabletLocations requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_IsAlterTableDone

kudu.master.MasterService.IsAlterTableDone RPC Rejections   Number of rejected kudu.master.MasterService.IsAlterTableDone requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_IsCreateTableDone

kudu.master.MasterService.IsCreateTableDone RPC Rejections   Number of rejected kudu.master.MasterService.IsCreateTableDone requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_ListMasters

kudu.master.MasterService.ListMasters RPC Rejections   Number of rejected kudu.master.MasterService.ListMasters requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_ListTables

kudu.master.MasterService.ListTables RPC Rejections   Number of rejected kudu.master.MasterService.ListTables requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_ListTabletServers

kudu.master.MasterService.ListTabletServers RPC Rejections   Number of rejected kudu.master.MasterService.ListTabletServers requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_Ping

kudu.master.MasterService.Ping RPC Rejections   Number of rejected kudu.master.MasterService.Ping requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_RefreshAuthzCache

kudu.master.MasterService.RefreshAuthzCache RPC Rejections   Number of rejected kudu.master.MasterService.RefreshAuthzCache requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_RemoveMaster

kudu.master.MasterService.RemoveMaster RPC Rejections   Number of rejected kudu.master.MasterService.RemoveMaster requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_ReplaceTablet

kudu.master.MasterService.ReplaceTablet RPC Rejections   Number of rejected kudu.master.MasterService.ReplaceTablet requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_master_MasterService_TSHeartbeat

kudu.master.MasterService.TSHeartbeat RPC Rejections   Number of rejected kudu.master.MasterService.TSHeartbeat requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_CheckLeaks

kudu.server.GenericService.CheckLeaks RPC Rejections   Number of rejected kudu.server.GenericService.CheckLeaks requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_DumpMemTrackers

kudu.server.GenericService.DumpMemTrackers RPC Rejections   Number of rejected kudu.server.GenericService.DumpMemTrackers requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_FlushCoverage

kudu.server.GenericService.FlushCoverage RPC Rejections   Number of rejected kudu.server.GenericService.FlushCoverage requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_GetFlags

kudu.server.GenericService.GetFlags RPC Rejections   Number of rejected kudu.server.GenericService.GetFlags requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_GetStatus

kudu.server.GenericService.GetStatus RPC Rejections   Number of rejected kudu.server.GenericService.GetStatus requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_ServerClock

kudu.server.GenericService.ServerClock RPC Rejections   Number of rejected kudu.server.GenericService.ServerClock requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_SetFlag

kudu.server.GenericService.SetFlag RPC Rejections   Number of rejected kudu.server.GenericService.SetFlag requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_SetServerWallClockForTests

kudu.server.GenericService.SetServerWallClockForTests RPC Rejections   Number of rejected kudu.server.GenericService.SetServerWallClockForTests requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_transactions_TxnManagerService_AbortTransaction

kudu.transactions.TxnManagerService.AbortTransaction RPC Rejections   Number of rejected kudu.transactions.TxnManagerService.AbortTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_transactions_TxnManagerService_BeginTransaction

kudu.transactions.TxnManagerService.BeginTransaction RPC Rejections   Number of rejected kudu.transactions.TxnManagerService.BeginTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_transactions_TxnManagerService_CommitTransaction

kudu.transactions.TxnManagerService.CommitTransaction RPC Rejections   Number of rejected kudu.transactions.TxnManagerService.CommitTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_transactions_TxnManagerService_GetTransactionState

kudu.transactions.TxnManagerService.GetTransactionState RPC Rejections   Number of rejected kudu.transactions.TxnManagerService.GetTransactionState requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_transactions_TxnManagerService_KeepTransactionAlive

kudu.transactions.TxnManagerService.KeepTransactionAlive RPC Rejections   Number of rejected kudu.transactions.TxnManagerService.KeepTransactionAlive requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_BeginTabletCopySession

kudu.tserver.TabletCopyService.BeginTabletCopySession RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.BeginTabletCopySession requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_CheckSessionActive

kudu.tserver.TabletCopyService.CheckSessionActive RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.CheckSessionActive requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_EndTabletCopySession

kudu.tserver.TabletCopyService.EndTabletCopySession RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.EndTabletCopySession requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_FetchData

kudu.tserver.TabletCopyService.FetchData RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.FetchData requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_AlterSchema

kudu.tserver.TabletServerAdminService.AlterSchema RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.AlterSchema requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_CoordinateTransaction

kudu.tserver.TabletServerAdminService.CoordinateTransaction RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.CoordinateTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_CreateTablet

kudu.tserver.TabletServerAdminService.CreateTablet RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.CreateTablet requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_DeleteTablet

kudu.tserver.TabletServerAdminService.DeleteTablet RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.DeleteTablet requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_ParticipateInTransaction

kudu.tserver.TabletServerAdminService.ParticipateInTransaction RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.ParticipateInTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_Quiesce

kudu.tserver.TabletServerAdminService.Quiesce RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.Quiesce requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Checksum

kudu.tserver.TabletServerService.Checksum RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Checksum requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_ListTablets

kudu.tserver.TabletServerService.ListTablets RPC Rejections   Number of rejected kudu.tserver.TabletServerService.ListTablets requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Ping

kudu.tserver.TabletServerService.Ping RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Ping requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Scan

kudu.tserver.TabletServerService.Scan RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Scan requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_ScannerKeepAlive

kudu.tserver.TabletServerService.ScannerKeepAlive RPC Rejections   Number of rejected kudu.tserver.TabletServerService.ScannerKeepAlive requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_SplitKeyRange

kudu.tserver.TabletServerService.SplitKeyRange RPC Rejections   Number of rejected kudu.tserver.TabletServerService.SplitKeyRange requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Write

kudu.tserver.TabletServerService.Write RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Write requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

ranger_server_inbound_queue_size_bytes

Ranger server inbound queue size (bytes)   Number of bytes in the inbound response queue of the Ranger server, recorded at the time a new response is read from the pipe and added to the inbound queue

Entity Type

server

Unit

bytes

Type

histogram

Level

info

 

ranger_server_inbound_queue_time_ms

Ranger server inbound queue time (ms)   Duration of time in ms spent in the Ranger server’s inbound response queue

Entity Type

server

Unit

milliseconds

Type

histogram

Level

info

 

ranger_server_outbound_queue_size_bytes

Ranger server outbound queue size (bytes)   Number of bytes in the outbound request queue of the Ranger server, recorded at the time a new request is added to the outbound request queue

Entity Type

server

Unit

bytes

Type

histogram

Level

info

 

ranger_server_outbound_queue_time_ms

Ranger server outbound queue time (ms)   Duration of time in ms spent in the Ranger server’s outbound request queue

Entity Type

server

Unit

milliseconds

Type

histogram

Level

info

 

ranger_subprocess_execution_time_ms

Ranger subprocess execution time (ms)   Duration of time in ms spent executing the Ranger subprocess request, excluding time spent spent in the subprocess queues

Entity Type

server

Unit

milliseconds

Type

histogram

Level

info

 

ranger_subprocess_inbound_queue_length

Ranger subprocess inbound queue length   Number of request messages in the Ranger subprocess' inbound request queue

Entity Type

server

Unit

messages

Type

histogram

Level

info

 

ranger_subprocess_inbound_queue_time_ms

Ranger subprocess inbound queue time (ms)   Duration of time in ms spent in the Ranger subprocess' inbound request queue

Entity Type

server

Unit

milliseconds

Type

histogram

Level

info

 

ranger_subprocess_outbound_queue_length

Ranger subprocess outbound queue length   Number of request messages in the Ranger subprocess' outbound response queue

Entity Type

server

Unit

messages

Type

histogram

Level

info

 

ranger_subprocess_outbound_queue_time_ms

Ranger subprocess outbound queue time (ms)   Duration of time in ms spent in the Ranger subprocess' outbound response queue

Entity Type

server

Unit

milliseconds

Type

histogram

Level

info

 

reactor_active_latency_us

Reactor Thread Active Latency   Histogram of the wall clock time for reactor thread wake-ups. The reactor thread is responsible for all network I/O and therefore outliers in this latency histogram directly contribute to the latency of both inbound and outbound RPCs.

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

reactor_load_percent

Reactor Thread Load Percentage   The percentage of time that the reactor is busy (not blocked awaiting network activity). If this metric shows significant samples nears 100%, increasing the number of reactors may be beneficial.

Entity Type

server

Unit

units

Type

histogram

Level

info

 

rows_deleted

Rows Deleted   Number of row delete operations performed on this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rows_inserted

Rows Inserted   Number of rows inserted into this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rows_updated

Rows Updated   Number of row update operations performed on this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rows_upserted

Rows Upserted   Number of rows upserted into this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rpc_connections_accepted

RPC Connections Accepted   Number of incoming TCP connections made to the RPC server

Entity Type

server

Unit

connections

Type

counter

Level

info

 

rpc_connections_accepted_unix_domain_socket

RPC Connections Accepted via UNIX Domain Socket   Number of incoming UNIX Domain Socket connections made to the RPC server

Entity Type

server

Unit

connections

Type

counter

Level

info

 

rpc_incoming_queue_time

RPC Queue Time   Number of microseconds incoming RPC requests spend in the worker queue

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

scanner_duration

Scanner Duration   Histogram of the duration of active scanners on this server

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

scanner_predicates_disabled

Scanner Column Predicates Disabled   Number of column predicates disabled during scan requests. This count measures the number of disableable column predicates like Bloom filter predicate that are automatically disabled if determined to be ineffective.

Entity Type

tablet

Unit

units

Type

counter

Level

info

 

state

Tablet State   State of this tablet.

Entity Type

tablet

Unit

state

Type

gauge

Level

info

 

tablet_copy_open_client_sessions

Open Table Copy Client Sessions   Number of currently open tablet copy client sessions on this server

Entity Type

server

Unit

sessions

Type

gauge

Level

info

 

tablet_copy_open_source_sessions

Open Table Copy Source Sessions   Number of currently open tablet copy source sessions on this server

Entity Type

server

Unit

sessions

Type

gauge

Level

info

 

tablets_num_bootstrapping

Number of Bootstrapping Tablets   Number of tablets currently bootstrapping

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_initialized

Number of Initialized Tablets   Number of tablets currently initialized

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_not_initialized

Number of Not Initialized Tablets   Number of tablets currently not initialized

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_running

Number of Running Tablets   Number of tablets currently running

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_shutdown

Number of Shut Down Tablets   Number of tablets currently shut down

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_stopped

Number of Stopped Tablets   Number of tablets currently stopped

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_stopping

Number of Stopping Tablets   Number of tablets currently stopping

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tcmalloc_current_total_thread_cache_bytes

Thread Cache Memory Usage   A measure of some of the memory TCMalloc is using (for small objects).

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

tcmalloc_max_total_thread_cache_bytes

Thread Cache Memory Limit   A limit to how much memory TCMalloc dedicates for small objects. Higher numbers trade off more memory use for — in some situations — improved efficiency.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

tcmalloc_pageheap_free_bytes

Free Heap Memory   Number of bytes in free, mapped pages in page heap. These bytes can be used to fulfill allocation requests. They always count towards virtual memory usage, and unless the underlying memory is swapped out by the OS, they also count towards physical memory usage.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

tcmalloc_pageheap_unmapped_bytes

Unmapped Heap Memory   Number of bytes in free, unmapped pages in page heap. These are bytes that have been released back to the OS, possibly by one of the MallocExtension "Release" calls. They can be used to fulfill allocation requests, but typically incur a page fault. They always count towards virtual memory usage, and depending on the OS, typically do not count towards physical memory usage.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

threads_running

Threads Running   Current number of running threads

Entity Type

server

Unit

threads

Type

gauge

Level

info

 

undo_delta_block_gc_delete_duration

Undo Delta Block GC Delete Duration   Time spent deleting ancient UNDO delta blocks.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

undo_delta_block_gc_init_duration

Undo Delta Block GC Init Duration   Time spent initializing ancient UNDO delta blocks.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

undo_delta_block_gc_perform_duration

Undo Delta Block GC Perform Duration   Time spent running the maintenance operation to GC ancient UNDO delta blocks.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

upserts_as_updates

Upserts converted into updates   Number of upserts which were applied as updates because the key already existed.

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

uptime

Server Uptime   Time interval since the server has started

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

voluntary_context_switches

Voluntary Context Switches   Total voluntary context switches

Entity Type

server

Unit

context switches

Type

counter

Level

info

 

Debug Metrics

Metrics tagged as 'debug' are diagnostically helpful but generally not monitored during normal operation.

all_transactions_inflight

Ops In Flight   Number of ops currently in-flight, including any type.

Entity Type

tablet

Unit

transactions

Type

gauge

Level

debug

 

alter_schema_transactions_inflight

Alter Schema Ops In Flight   Number of alter schema ops currently in-flight

Entity Type

tablet

Unit

transactions

Type

gauge

Level

debug

 

block_cache_evictions

Block Cache Evictions   Number of blocks evicted from the cache

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_hits

Block Cache Hits   Number of lookups that found a block

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_hits_caching

Block Cache Hits (Caching)   Number of lookups that were expecting a block that found one.Use this number instead of cache_hits when trying to determine how efficient the cache is

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_inserts

Block Cache Inserts   Number of blocks inserted in the cache

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_lookups

Block Cache Lookups   Number of blocks looked up from the cache

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_misses

Block Cache Misses   Number of lookups that didn’t yield a block

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_misses_caching

Block Cache Misses (Caching)   Number of lookups that were expecting a block that didn’t yield one.Use this number instead of cache_misses when trying to determine how efficient the cache is

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_blocks_created

Data Blocks Created   Number of data blocks that were created since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_blocks_deleted

Data Blocks Deleted   Number of data blocks that were deleted since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_bytes_read

Block Data Bytes Read   Number of bytes of block data read since service start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

block_manager_total_bytes_written

Block Data Bytes Written   Number of bytes of block data written since service start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

block_manager_total_disk_sync

Block Data Disk Synchronization Count   Number of disk synchronizations of block data since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_readable_blocks

Data Blocks Opened For Read   Number of data blocks opened for reading since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_writable_blocks

Data Blocks Opened For Write   Number of data blocks opened for writing since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

bloom_lookups

Bloom Filter Lookups   Number of times a bloom filter was consulted

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

bloom_lookups_per_op

Bloom Lookups per Operation   Tracks the number of bloom filter lookups performed by each operation. A single operation may perform several bloom filter lookups if the tablet is not fully compacted. High frequency of high values may indicate that compaction is falling behind.

Entity Type

tablet

Unit

probes

Type

histogram

Level

debug

 

builtin_ntp_max_errors

Built-In NTP Maximum Time Errors   Statistics on the maximum true time error computed by built-in NTP client

Entity Type

server

Unit

microseconds

Type

histogram

Level

debug

 

builtin_ntp_time

Built-in NTP Time   Latest true time as tracked by built-in NTP client

Entity Type

server

Unit

microseconds

Type

gauge

Level

debug

 

bytes_flushed

Bytes Flushed   Amount of data that has been flushed to disk by this tablet.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

code_cache_hits

Codegen Cache Hits   Number of codegen cache hits since start

Entity Type

server

Unit

hits

Type

counter

Level

debug

 

code_cache_queries

Codegen Cache Queries   Number of codegen cache queries (hits + misses) since start

Entity Type

server

Unit

queries

Type

counter

Level

debug

 

commit_wait_duration

Commit-Wait Duration   Time spent waiting for COMMIT_WAIT external consistency writes for this tablet.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

compact_rs_running

RowSet Compactions Running   Number of RowSet compactions currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

delete_ignore_errors

Delete Ignore Errors   Number of delete ignore operations for this tablet which were ignored due to an error since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

deleted_rowset_estimated_retained_bytes

Estimated Deletable Bytes Retained in Deleted Rowsets   Estimated bytes of deletable data in deleted rowsets for this tablet.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

debug

 

deleted_rowset_gc_bytes_deleted

Deleted Rowsets GC Bytes Deleted   Number of bytes deleted by garbage-collecting deleted rowsets.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

deleted_rowset_gc_running

Deleted Rowset GC Running   Number of deleted rowset GC operations currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

delta_file_lookups

Delta File Lookups   Number of times a delta file was consulted

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

delta_major_compact_rs_running

Major Delta Compactions Running   Number of delta major compactions currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

delta_minor_compact_rs_running

Minor Delta Compactions Running   Number of delta minor compactions currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

file_cache_evictions

File Cache Evictions   Number of file descriptors evicted from the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_hits

File Cache Hits   Number of lookups that found a file descriptor

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_hits_caching

File Cache Hits (Caching)   Number of lookups that were expecting a file descriptor that found one. Use this number instead of cache_hits when trying to determine how efficient the cache is

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_inserts

File Cache Inserts   Number of file descriptors inserted in the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_lookups

File Cache Lookups   Number of file descriptors looked up from the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_misses

File Cache Misses   Number of lookups that didn’t yield a file descriptor

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_misses_caching

File Cache Misses (Caching)   Number of lookups that were expecting a file descriptor that didn’t yield one. Use this number instead of cache_misses when trying to determine how efficient the cache is

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

flush_dms_running

DeltaMemStore Flushes Running   Number of delta memstore flushes currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

flush_mrs_running

MemRowSet Flushes Running   Number of MemRowSet flushes currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

glog_info_messages

INFO-level Log Messages   Number of INFO-level log messages emitted by the application.

Entity Type

server

Unit

messages

Type

counter

Level

debug

 

hybrid_clock_max_errors

Hybrid Clock Maximum Errors   The statistics on the maximum error of the underlying clock

Entity Type

server

Unit

microseconds

Type

histogram

Level

debug

 

in_progress_ops

Operations in Progress   Number of operations in the peer’s queue ack’d by a minority of peers.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

insert_ignore_errors

Insert Ignore Errors   Number of insert ignore operations for this tablet which were ignored due to an error since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

insertions_failed_dup_key

Duplicate Key Inserts   Number of inserts which failed because the key already existed

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

key_file_lookups

Key File Lookups   Number of times a key cfile was consulted

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

key_file_lookups_per_op

Key Lookups per Operation   Tracks the number of key file lookups performed by each operation. A single operation may perform several key file lookups if the tablet is not fully compacted and if bloom filters are not effectively culling lookups.

Entity Type

tablet

Unit

probes

Type

histogram

Level

debug

 

last_read_elapsed_time

Seconds Since Last Read   The elapsed time, in seconds, since the last read operation on this tablet, or since this Tablet object was created on current tserver if it hasn’t been read since then.

Entity Type

tablet

Unit

seconds

Type

gauge

Level

debug

 

last_write_elapsed_time

Seconds Since Last Write   The elapsed time, in seconds, since the last write operation on this tablet, or since this Tablet object was created on current tserver if it hasn’t been written to since then.

Entity Type

tablet

Unit

seconds

Type

gauge

Level

debug

 

location_mapping_cache_hits

Location Mapping Cache Hits   Number of times location mapping assignment used cached data

Entity Type

server

Unit

hits

Type

counter

Level

debug

 

location_mapping_cache_queries

Location Mapping Cache Queries   Number of queries to the location mapping cache

Entity Type

server

Unit

queries

Type

counter

Level

debug

 

log_block_manager_dead_containers_deleted

Number of Dead Block Containers Deleted   Number of full (but dead) block containers that were deleted

Entity Type

server

Unit

log block containers

Type

counter

Level

debug

 

log_block_manager_holes_punched

Number of Holes Punched   Number of holes punched since service start

Entity Type

server

Unit

holes

Type

counter

Level

debug

 

log_bytes_logged

Bytes Written to WAL   Number of bytes logged since service start

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

log_cache_num_ops

Log Cache Operation Count   Number of operations in the log cache.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

log_cache_size

Log Cache Memory Usage   Amount of memory in use for caching the local log.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

debug

 

log_entry_batches_per_group

Log Group Commit Batch Size   Number of log entry batches in a group commit group

Entity Type

tablet

Unit

requests

Type

histogram

Level

debug

 

log_group_commit_latency

Log Group Commit Latency   Microseconds spent on committing an entire group

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

log_reader_bytes_read

Bytes Read From Log   Data read from the WAL since tablet start

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

log_reader_entries_read

Entries Read From Log   Number of entries read from the WAL since tablet start

Entity Type

tablet

Unit

entries

Type

counter

Level

debug

 

log_roll_latency

Log Roll Latency   Microseconds spent on rolling over to a new log segment file

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

majority_done_ops

Leader Operations Acked by Majority   Number of operations in the leader queue ack’d by a majority but not all peers. This metric is always zero for followers.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

mrs_lookups

MemRowSet Lookups   Number of times a MemRowSet was consulted.

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

raft_term

Current Raft Consensus Term   Current Term of the Raft Consensus algorithm. This number increments each time a leader election is started.

Entity Type

tablet

Unit

units

Type

gauge

Level

debug

 

scanner_bytes_returned

Scanner Bytes Returned   Number of bytes returned by scanners to clients. This count is measured after predicates are applied and the data is decoded for consumption by clients, and thus is not a reflection of the amount of work being done by scanners.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

scanner_bytes_scanned_from_disk

Scanner Bytes Scanned From Disk   Number of bytes read by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual IO that has been caused by scan operations compared to the Scanner Bytes Returned metric. Note that this only counts data that has been flushed to disk, and does not include data read from in-memory stores. However, itincludes both cache misses and cache hits.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

scanner_cells_returned

Scanner Cells Returned   Number of table cells returned by scanners to clients. This count is measured after predicates are applied, and thus is not a reflection of the amount of work being done by scanners.

Entity Type

tablet

Unit

cells

Type

counter

Level

debug

 

scanner_cells_scanned_from_disk

Scanner Cells Scanned From Disk   Number of table cells processed by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual table cells that have been processed by scan operations compared to the Scanner Cells Returned metric. Note that this only counts data that has been flushed to disk, and does not include data read from in-memory stores. However, itincludes both cache misses and cache hits.

Entity Type

tablet

Unit

cells

Type

counter

Level

debug

 

scanner_rows_returned

Scanner Rows Returned   Number of rows returned by scanners to clients. This count is measured after predicates are applied, and thus is not a reflection of the amount of work being done by scanners.

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

scanner_rows_scanned

Scanner Rows Scanned   Number of rows processed by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual table rows that have been processed by scan operations compared to the Scanner Rows Returned metric.

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

scans_started

Scans Started   Number of scanners which have been started on this tablet

Entity Type

tablet

Unit

scanners

Type

counter

Level

debug

 

snapshot_read_inflight_wait_duration

Time Waiting For Snapshot Reads   Time spent waiting for in-flight writes to complete for READ_AT_SNAPSHOT scans.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

table_locations_cache_evictions

Table Locations Cache Evictions   Number of entries evicted from the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

table_locations_cache_hits

Table Locations Cache Hits   Number of lookups that found a cached entry

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

table_locations_cache_inserts

Table Locations Cache Inserts   Number of entries inserted in the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

table_locations_cache_lookups

Table Locations Cache Lookups   Number of entries looked up from the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

table_locations_cache_memory_usage

Table Locations Cache Memory Usage   Memory consumed by the cache

Entity Type

server

Unit

bytes

Type

gauge

Level

debug

 

table_locations_cache_misses

Table Locations Cache Misses   Number of lookups that didn’t find a cached entry

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

tablet_active_scanners

Active Scanners   Number of scanners that are currently active on this tablet

Entity Type

tablet

Unit

scanners

Type

gauge

Level

debug

 

tablet_copy_bytes_fetched

Bytes Fetched By Tablet Copy   Number of bytes fetched during tablet copy operations since server start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

tablet_copy_bytes_sent

Bytes Sent For Tablet Copy   Number of bytes sent during tablet copy operations since server start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

threads_started

Threads Started   Total number of threads started on this server

Entity Type

server

Unit

threads

Type

counter

Level

debug

 

time_since_last_leader_heartbeat

Time Since Last Leader Heartbeat   The time elapsed since the last heartbeat from the leader in milliseconds. This metric is identically zero on a leader replica.

Entity Type

tablet

Unit

milliseconds

Type

gauge

Level

debug

 

undo_delta_block_estimated_retained_bytes

Estimated Deletable Bytes Retained in Undo Delta Blocks   Estimated bytes of deletable data in undo delta blocks for this tablet. May be an overestimate.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

debug

 

undo_delta_block_gc_bytes_deleted

Undo Delta Block GC Bytes Deleted   Number of bytes deleted by garbage-collecting old UNDO delta blocks on this tablet since this server was restarted. Does not include bytes garbage collected during compactions.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

undo_delta_block_gc_running

Undo Delta Block GC Running   Number of UNDO delta block GC operations currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

update_ignore_errors

Update Ignore Errors   Number of update ignore operations for this tablet which were ignored due to an error since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

write_op_duration_client_propagated_consistency

Write Op Duration with Propagated Consistency   Duration of writes to this tablet with external consistency set to CLIENT_PROPAGATED.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

write_op_duration_commit_wait_consistency

Write Op Duration with Commit-Wait Consistency   Duration of writes to this tablet with external consistency set to COMMIT_WAIT.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

write_transactions_inflight

Write Ops In Flight   Number of write ops currently in-flight

Entity Type

tablet

Unit

transactions

Type

gauge

Level

debug

 


kudu-tserver Metrics

Warning Metrics

Metrics tagged as 'warn' are metrics which can often indicate operational oddities that may need more investigation.

data_dirs_failed

Data Directories Failed   Number of data directories whose disks are currently in a failed state

Entity Type

server

Unit

data directories

Type

gauge

Level

warn

 

data_dirs_full

Data Directories Full   Number of data directories whose disks are currently full

Entity Type

server

Unit

data directories

Type

gauge

Level

warn

 

failed_elections_since_stable_leader

Failed Elections Since Stable Leader   Number of failed elections on this node since there was a stable leader. This number increments on each failed election and resets on each successful one.

Entity Type

tablet

Unit

units

Type

gauge

Level

warn

 

follower_memory_pressure_rejections

Follower Memory Pressure Rejections   Number of RPC requests rejected due to memory pressure while FOLLOWER.

Entity Type

tablet

Unit

requests

Type

counter

Level

warn

 

glog_error_messages

ERROR-level Log Messages   Number of ERROR-level log messages emitted by the application.

Entity Type

server

Unit

messages

Type

counter

Level

warn

 

glog_warning_messages

WARNING-level Log Messages   Number of WARNING-level log messages emitted by the application.

Entity Type

server

Unit

messages

Type

counter

Level

warn

 

hybrid_clock_extrapolating

Hybrid Clock Is Being Extrapolated   Whether HybridClock timestamps are extrapolated because of inability to read the underlying clock

Entity Type

server

Unit

state

Type

gauge

Level

warn

 

hybrid_clock_extrapolation_intervals

Intervals of Hybrid Clock Extrapolation   The statistics on the duration of intervals when the underlying clock was extrapolated instead of using the direct readings

Entity Type

server

Unit

seconds

Type

histogram

Level

warn

 

leader_memory_pressure_rejections

Leader Memory Pressure Rejections   Number of RPC requests rejected due to memory pressure while LEADER.

Entity Type

tablet

Unit

requests

Type

counter

Level

warn

 

maintenance_op_prepare_failed

Number Of Operations With Failed Prepare()   Number of times when calling Prepare() on a maintenance operation failed

Entity Type

server

Unit

operations

Type

counter

Level

warn

 

op_apply_queue_length

Operation Apply Queue Length   Number of operations waiting to be applied to the tablet. High queue lengths indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

server

Unit

tasks

Type

histogram

Level

warn

 

op_apply_queue_overload_rejections

Number of Rejected Write Requests Due to Queue Overloaded Error   Number of rejected write requests due to overloaded op apply queue

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

op_apply_queue_time

Operation Apply Queue Time   Time that operations spent waiting in the apply queue before being processed. High queue times indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

server

Unit

microseconds

Type

histogram

Level

warn

 

op_apply_run_time

Operation Apply Run Time   Time that operations spent being applied to the tablet. High values may indicate that the server is under-provisioned or that operations consist of very large batches.

Entity Type

server

Unit

microseconds

Type

histogram

Level

warn

 

ops_behind_leader

Operations Behind Leader   Number of operations this server believes it is behind the leader.

Entity Type

tablet

Unit

operations

Type

gauge

Level

warn

 

rpcs_queue_overflow

RPC Queue Overflows   Number of RPCs dropped because the service queue was full.

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

rpcs_timed_out_in_queue

RPC Queue Timeouts   Number of RPCs whose timeout elapsed while waiting in the service queue, and thus were not processed.

Entity Type

server

Unit

requests

Type

counter

Level

warn

 

scanners_expired

Scanners Expired   Number of scanners that have expired due to inactivity since service start

Entity Type

server

Unit

scanners

Type

counter

Level

warn

 

spinlock_contention_time

Spinlock Contention Time   Amount of time consumed by contention on internal spinlocks since the server started. If this increases rapidly, it may indicate a performance issue in Kudu internals triggered by a particular workload and warrant investigation.

Entity Type

server

Unit

microseconds

Type

counter

Level

warn

 

tablets_num_failed

Number of Failed Tablets   Number of failed tablets

Entity Type

server

Unit

tablets

Type

gauge

Level

warn

 

transaction_memory_limit_rejections

Tablet Op Memory Limit Rejections   Number of ops rejected because the tablet’s op memory limit was reached.

Entity Type

tablet

Unit

transactions

Type

counter

Level

warn

 

transaction_memory_pressure_rejections

Op Memory Pressure Rejections   Number of ops rejected because the tablet’s opmemory usage exceeds the op memory limit or the limitof an ancestral tracker.

Entity Type

tablet

Unit

transactions

Type

counter

Level

warn

 

Info Metrics

Metrics tagged as 'info' are generally useful metrics that operators always want to have available but may not be monitored under normal circumstances.

active_scanners

Active Scanners   Number of scanners that are currently active

Entity Type

server

Unit

scanners

Type

gauge

Level

info

 

average_diskrowset_height

Average DiskRowSet Height   Average height of the diskrowsets in this tablet replica. The larger the average height, the more uncompacted the tablet replica is.

Entity Type

tablet

Unit

units

Type

gauge

Level

info

 

block_cache_usage

Block Cache Memory Usage   Memory consumed by the block cache

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

block_manager_blocks_open_reading

Data Blocks Open For Read   Number of data blocks currently open for reading

Entity Type

server

Unit

blocks

Type

gauge

Level

info

 

block_manager_blocks_open_writing

Data Blocks Open For Write   Number of data blocks currently open for writing

Entity Type

server

Unit

blocks

Type

gauge

Level

info

 

builtin_ntp_error

Built-in NTP Latest Maximum Time Error   Latest maximum time error as tracked by built-in NTP client

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

builtin_ntp_local_clock_delta

Local Clock vs Built-In NTP True Time Delta   Delta between local clock and true time tracked by built-in NTP client; set to 2^63-1 when true time is not tracked

Entity Type

server

Unit

milliseconds

Type

gauge

Level

info

 

compact_rs_duration

RowSet Compaction Duration   Time spent compacting RowSets.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

cpu_stime

System CPU Time   Total system CPU time of the process

Entity Type

server

Unit

milliseconds

Type

counter

Level

info

 

cpu_utime

User CPU Time   Total user CPU time of the process

Entity Type

server

Unit

milliseconds

Type

counter

Level

info

 

deleted_rowset_gc_duration

Deleted Rowset GC Duration   Time spent running the maintenance operation to GC deleted rowsets.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

delta_file_lookups_per_op

Delta File Lookups per Operation   Tracks the number of delta file lookups performed by each operation. A single operation may perform several delta file lookups if the tablet is not fully compacted. High frequency of high values may indicate that compaction is falling behind.

Entity Type

tablet

Unit

probes

Type

histogram

Level

info

 

delta_major_compact_rs_duration

Major Delta Compaction Duration   Seconds spent major delta compacting.

Entity Type

tablet

Unit

seconds

Type

histogram

Level

info

 

delta_minor_compact_rs_duration

Minor Delta Compaction Duration   Time spent minor delta compacting.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

file_cache_usage

File Cache Usage   Number of entries in the file cache

Entity Type

server

Unit

entries

Type

gauge

Level

info

 

flush_dms_duration

DeltaMemStore Flush Duration   Time spent flushing DeltaMemStores.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

flush_mrs_duration

MemRowSet Flush Duration   Time spent flushing MemRowSets.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

generic_current_allocated_bytes

Heap Memory Usage   Number of bytes used by the application. This will not typically match the memory use reported by the OS, because it does not include TCMalloc overhead or memory fragmentation.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

generic_heap_size

Reserved Heap Memory   Bytes of system memory reserved by TCMalloc.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

handler_latency_kudu_consensus_ConsensusService_BulkChangeConfig

kudu.consensus.ConsensusService.BulkChangeConfig RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.BulkChangeConfig RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_ChangeConfig

kudu.consensus.ConsensusService.ChangeConfig RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.ChangeConfig RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_GetConsensusState

kudu.consensus.ConsensusService.GetConsensusState RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.GetConsensusState RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_GetLastOpId

kudu.consensus.ConsensusService.GetLastOpId RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.GetLastOpId RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_GetNodeInstance

kudu.consensus.ConsensusService.GetNodeInstance RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.GetNodeInstance RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_LeaderStepDown

kudu.consensus.ConsensusService.LeaderStepDown RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.LeaderStepDown RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_RequestConsensusVote

kudu.consensus.ConsensusService.RequestConsensusVote RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.RequestConsensusVote RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_RunLeaderElection

kudu.consensus.ConsensusService.RunLeaderElection RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.RunLeaderElection RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_StartTabletCopy

kudu.consensus.ConsensusService.StartTabletCopy RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.StartTabletCopy RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_UnsafeChangeConfig

kudu.consensus.ConsensusService.UnsafeChangeConfig RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.UnsafeChangeConfig RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_consensus_ConsensusService_UpdateConsensus

kudu.consensus.ConsensusService.UpdateConsensus RPC Time   Microseconds spent handling kudu.consensus.ConsensusService.UpdateConsensus RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_CheckLeaks

kudu.server.GenericService.CheckLeaks RPC Time   Microseconds spent handling kudu.server.GenericService.CheckLeaks RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_DumpMemTrackers

kudu.server.GenericService.DumpMemTrackers RPC Time   Microseconds spent handling kudu.server.GenericService.DumpMemTrackers RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_FlushCoverage

kudu.server.GenericService.FlushCoverage RPC Time   Microseconds spent handling kudu.server.GenericService.FlushCoverage RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_GetFlags

kudu.server.GenericService.GetFlags RPC Time   Microseconds spent handling kudu.server.GenericService.GetFlags RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_GetStatus

kudu.server.GenericService.GetStatus RPC Time   Microseconds spent handling kudu.server.GenericService.GetStatus RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_ServerClock

kudu.server.GenericService.ServerClock RPC Time   Microseconds spent handling kudu.server.GenericService.ServerClock RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_SetFlag

kudu.server.GenericService.SetFlag RPC Time   Microseconds spent handling kudu.server.GenericService.SetFlag RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_server_GenericService_SetServerWallClockForTests

kudu.server.GenericService.SetServerWallClockForTests RPC Time   Microseconds spent handling kudu.server.GenericService.SetServerWallClockForTests RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_BeginTabletCopySession

kudu.tserver.TabletCopyService.BeginTabletCopySession RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.BeginTabletCopySession RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_CheckSessionActive

kudu.tserver.TabletCopyService.CheckSessionActive RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.CheckSessionActive RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_EndTabletCopySession

kudu.tserver.TabletCopyService.EndTabletCopySession RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.EndTabletCopySession RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletCopyService_FetchData

kudu.tserver.TabletCopyService.FetchData RPC Time   Microseconds spent handling kudu.tserver.TabletCopyService.FetchData RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_AlterSchema

kudu.tserver.TabletServerAdminService.AlterSchema RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.AlterSchema RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_CoordinateTransaction

kudu.tserver.TabletServerAdminService.CoordinateTransaction RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.CoordinateTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_CreateTablet

kudu.tserver.TabletServerAdminService.CreateTablet RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.CreateTablet RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet

kudu.tserver.TabletServerAdminService.DeleteTablet RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.DeleteTablet RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_ParticipateInTransaction

kudu.tserver.TabletServerAdminService.ParticipateInTransaction RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.ParticipateInTransaction RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerAdminService_Quiesce

kudu.tserver.TabletServerAdminService.Quiesce RPC Time   Microseconds spent handling kudu.tserver.TabletServerAdminService.Quiesce RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Checksum

kudu.tserver.TabletServerService.Checksum RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Checksum RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_ListTablets

kudu.tserver.TabletServerService.ListTablets RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.ListTablets RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Ping

kudu.tserver.TabletServerService.Ping RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Ping RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Scan

kudu.tserver.TabletServerService.Scan RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Scan RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_ScannerKeepAlive

kudu.tserver.TabletServerService.ScannerKeepAlive RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.ScannerKeepAlive RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_SplitKeyRange

kudu.tserver.TabletServerService.SplitKeyRange RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.SplitKeyRange RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

handler_latency_kudu_tserver_TabletServerService_Write

kudu.tserver.TabletServerService.Write RPC Time   Microseconds spent handling kudu.tserver.TabletServerService.Write RPC requests

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

hybrid_clock_error

Hybrid Clock Error   Server clock maximum error; returns 2^64-1 when unable to read the underlying clock

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

hybrid_clock_timestamp

Hybrid Clock Timestamp   Hybrid clock timestamp; returns 2^64-1 when unable to read the underlying clock

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

involuntary_context_switches

Involuntary Context Switches   Total involuntary context switches

Entity Type

server

Unit

context switches

Type

counter

Level

info

 

live_row_count

Tablet Live Row Count   Number of live rows in this tablet, excludes deleted rows.

Entity Type

tablet

Unit

rows

Type

gauge

Level

info

 

log_append_latency

Log Append Latency   Microseconds spent on appending to the log segment file

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

log_block_manager_blocks_under_management

Blocks Under Management   Number of data blocks currently under management

Entity Type

server

Unit

blocks

Type

gauge

Level

info

 

log_block_manager_bytes_under_management

Bytes Under Management   Number of bytes of data blocks currently under management

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

log_block_manager_containers

Number of Block Containers   Number of log block containers

Entity Type

server

Unit

log block containers

Type

gauge

Level

info

 

log_block_manager_full_containers

Number of Full Block Containers   Number of full log block containers

Entity Type

server

Unit

log block containers

Type

gauge

Level

info

 

log_gc_duration

Log GC Duration   Time spent garbage collecting the logs.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

log_gc_running

Log GCs Running   Number of log GC operations currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

info

 

log_reader_read_batch_latency

Log Read Latency   Microseconds spent reading log entry batches

Entity Type

tablet

Unit

bytes

Type

histogram

Level

info

 

log_sync_latency

Log Sync Latency   Microseconds spent on synchronizing the log segment file

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

logical_clock_timestamp

Logical Clock Timestamp   Logical clock timestamp.

Entity Type

server

Unit

units

Type

gauge

Level

info

 

maintenance_op_find_best_candidate_duration

Time Taken To Find Best Maintenance Operation   Time spent choosing a maintenance operation with highest scores among available candidates

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

memrowset_size

MemRowSet Memory Usage   Size of this tablet’s memrowset

Entity Type

tablet

Unit

bytes

Type

gauge

Level

info

 

merged_entities_count_of_server

Entities Count Merged From   Count of entities merged together when entities are merged by common attribute value.

Entity Type

server

Unit

entries

Type

gauge

Level

info

 

merged_entities_count_of_tablet

Entities Count Merged From   Count of entities merged together when entities are merged by common attribute value.

Entity Type

tablet

Unit

entries

Type

gauge

Level

info

 

num_raft_leaders

Number of Raft Leaders   Number of tablet replicas that are Raft leaders

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

num_rowsets_on_disk

Tablet Number of Rowsets on Disk   Number of diskrowsets in this tablet

Entity Type

tablet

Unit

units

Type

gauge

Level

info

 

on_disk_data_size

Tablet Data Size On Disk   Space used by this tablet’s data blocks.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

info

 

on_disk_size

Tablet Size On Disk   Space used by this tablet on disk, including metadata.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

info

 

op_prepare_queue_length

Operation Prepare Queue Length   Number of operations waiting to be prepared within this tablet. High queue lengths indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

tablet

Unit

tasks

Type

histogram

Level

info

 

op_prepare_queue_time

Operation Prepare Queue Time   Time that operations spent waiting in the prepare queue before being processed. High queue times indicate that the server is unable to process operations as fast as they are being written to the WAL.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

op_prepare_run_time

Operation Prepare Run Time   Time that operations spent being prepared in the tablet. High values may indicate that the server is under-provisioned or that operations are experiencing high contention with one another for locks.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_BulkChangeConfig

kudu.consensus.ConsensusService.BulkChangeConfig RPC Rejections   Number of rejected kudu.consensus.ConsensusService.BulkChangeConfig requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_ChangeConfig

kudu.consensus.ConsensusService.ChangeConfig RPC Rejections   Number of rejected kudu.consensus.ConsensusService.ChangeConfig requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_GetConsensusState

kudu.consensus.ConsensusService.GetConsensusState RPC Rejections   Number of rejected kudu.consensus.ConsensusService.GetConsensusState requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_GetLastOpId

kudu.consensus.ConsensusService.GetLastOpId RPC Rejections   Number of rejected kudu.consensus.ConsensusService.GetLastOpId requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_GetNodeInstance

kudu.consensus.ConsensusService.GetNodeInstance RPC Rejections   Number of rejected kudu.consensus.ConsensusService.GetNodeInstance requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_LeaderStepDown

kudu.consensus.ConsensusService.LeaderStepDown RPC Rejections   Number of rejected kudu.consensus.ConsensusService.LeaderStepDown requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_RequestConsensusVote

kudu.consensus.ConsensusService.RequestConsensusVote RPC Rejections   Number of rejected kudu.consensus.ConsensusService.RequestConsensusVote requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_RunLeaderElection

kudu.consensus.ConsensusService.RunLeaderElection RPC Rejections   Number of rejected kudu.consensus.ConsensusService.RunLeaderElection requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_StartTabletCopy

kudu.consensus.ConsensusService.StartTabletCopy RPC Rejections   Number of rejected kudu.consensus.ConsensusService.StartTabletCopy requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_UnsafeChangeConfig

kudu.consensus.ConsensusService.UnsafeChangeConfig RPC Rejections   Number of rejected kudu.consensus.ConsensusService.UnsafeChangeConfig requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_consensus_ConsensusService_UpdateConsensus

kudu.consensus.ConsensusService.UpdateConsensus RPC Rejections   Number of rejected kudu.consensus.ConsensusService.UpdateConsensus requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_CheckLeaks

kudu.server.GenericService.CheckLeaks RPC Rejections   Number of rejected kudu.server.GenericService.CheckLeaks requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_DumpMemTrackers

kudu.server.GenericService.DumpMemTrackers RPC Rejections   Number of rejected kudu.server.GenericService.DumpMemTrackers requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_FlushCoverage

kudu.server.GenericService.FlushCoverage RPC Rejections   Number of rejected kudu.server.GenericService.FlushCoverage requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_GetFlags

kudu.server.GenericService.GetFlags RPC Rejections   Number of rejected kudu.server.GenericService.GetFlags requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_GetStatus

kudu.server.GenericService.GetStatus RPC Rejections   Number of rejected kudu.server.GenericService.GetStatus requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_ServerClock

kudu.server.GenericService.ServerClock RPC Rejections   Number of rejected kudu.server.GenericService.ServerClock requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_SetFlag

kudu.server.GenericService.SetFlag RPC Rejections   Number of rejected kudu.server.GenericService.SetFlag requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_server_GenericService_SetServerWallClockForTests

kudu.server.GenericService.SetServerWallClockForTests RPC Rejections   Number of rejected kudu.server.GenericService.SetServerWallClockForTests requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_BeginTabletCopySession

kudu.tserver.TabletCopyService.BeginTabletCopySession RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.BeginTabletCopySession requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_CheckSessionActive

kudu.tserver.TabletCopyService.CheckSessionActive RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.CheckSessionActive requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_EndTabletCopySession

kudu.tserver.TabletCopyService.EndTabletCopySession RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.EndTabletCopySession requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletCopyService_FetchData

kudu.tserver.TabletCopyService.FetchData RPC Rejections   Number of rejected kudu.tserver.TabletCopyService.FetchData requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_AlterSchema

kudu.tserver.TabletServerAdminService.AlterSchema RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.AlterSchema requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_CoordinateTransaction

kudu.tserver.TabletServerAdminService.CoordinateTransaction RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.CoordinateTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_CreateTablet

kudu.tserver.TabletServerAdminService.CreateTablet RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.CreateTablet requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_DeleteTablet

kudu.tserver.TabletServerAdminService.DeleteTablet RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.DeleteTablet requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_ParticipateInTransaction

kudu.tserver.TabletServerAdminService.ParticipateInTransaction RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.ParticipateInTransaction requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerAdminService_Quiesce

kudu.tserver.TabletServerAdminService.Quiesce RPC Rejections   Number of rejected kudu.tserver.TabletServerAdminService.Quiesce requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Checksum

kudu.tserver.TabletServerService.Checksum RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Checksum requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_ListTablets

kudu.tserver.TabletServerService.ListTablets RPC Rejections   Number of rejected kudu.tserver.TabletServerService.ListTablets requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Ping

kudu.tserver.TabletServerService.Ping RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Ping requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Scan

kudu.tserver.TabletServerService.Scan RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Scan requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_ScannerKeepAlive

kudu.tserver.TabletServerService.ScannerKeepAlive RPC Rejections   Number of rejected kudu.tserver.TabletServerService.ScannerKeepAlive requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_SplitKeyRange

kudu.tserver.TabletServerService.SplitKeyRange RPC Rejections   Number of rejected kudu.tserver.TabletServerService.SplitKeyRange requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

queue_overflow_rejections_kudu_tserver_TabletServerService_Write

kudu.tserver.TabletServerService.Write RPC Rejections   Number of rejected kudu.tserver.TabletServerService.Write requests due to RPC queue overflow

Entity Type

server

Unit

requests

Type

counter

Level

info

 

reactor_active_latency_us

Reactor Thread Active Latency   Histogram of the wall clock time for reactor thread wake-ups. The reactor thread is responsible for all network I/O and therefore outliers in this latency histogram directly contribute to the latency of both inbound and outbound RPCs.

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

reactor_load_percent

Reactor Thread Load Percentage   The percentage of time that the reactor is busy (not blocked awaiting network activity). If this metric shows significant samples nears 100%, increasing the number of reactors may be beneficial.

Entity Type

server

Unit

units

Type

histogram

Level

info

 

rows_deleted

Rows Deleted   Number of row delete operations performed on this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rows_inserted

Rows Inserted   Number of rows inserted into this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rows_updated

Rows Updated   Number of row update operations performed on this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rows_upserted

Rows Upserted   Number of rows upserted into this tablet since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

rpc_connections_accepted

RPC Connections Accepted   Number of incoming TCP connections made to the RPC server

Entity Type

server

Unit

connections

Type

counter

Level

info

 

rpc_connections_accepted_unix_domain_socket

RPC Connections Accepted via UNIX Domain Socket   Number of incoming UNIX Domain Socket connections made to the RPC server

Entity Type

server

Unit

connections

Type

counter

Level

info

 

rpc_incoming_queue_time

RPC Queue Time   Number of microseconds incoming RPC requests spend in the worker queue

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

scanner_duration

Scanner Duration   Histogram of the duration of active scanners on this server

Entity Type

server

Unit

microseconds

Type

histogram

Level

info

 

scanner_predicates_disabled

Scanner Column Predicates Disabled   Number of column predicates disabled during scan requests. This count measures the number of disableable column predicates like Bloom filter predicate that are automatically disabled if determined to be ineffective.

Entity Type

tablet

Unit

units

Type

counter

Level

info

 

state

Tablet State   State of this tablet.

Entity Type

tablet

Unit

state

Type

gauge

Level

info

 

tablet_copy_open_client_sessions

Open Table Copy Client Sessions   Number of currently open tablet copy client sessions on this server

Entity Type

server

Unit

sessions

Type

gauge

Level

info

 

tablet_copy_open_source_sessions

Open Table Copy Source Sessions   Number of currently open tablet copy source sessions on this server

Entity Type

server

Unit

sessions

Type

gauge

Level

info

 

tablets_num_bootstrapping

Number of Bootstrapping Tablets   Number of tablets currently bootstrapping

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_initialized

Number of Initialized Tablets   Number of tablets currently initialized

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_not_initialized

Number of Not Initialized Tablets   Number of tablets currently not initialized

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_running

Number of Running Tablets   Number of tablets currently running

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_shutdown

Number of Shut Down Tablets   Number of tablets currently shut down

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_stopped

Number of Stopped Tablets   Number of tablets currently stopped

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tablets_num_stopping

Number of Stopping Tablets   Number of tablets currently stopping

Entity Type

server

Unit

tablets

Type

gauge

Level

info

 

tcmalloc_current_total_thread_cache_bytes

Thread Cache Memory Usage   A measure of some of the memory TCMalloc is using (for small objects).

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

tcmalloc_max_total_thread_cache_bytes

Thread Cache Memory Limit   A limit to how much memory TCMalloc dedicates for small objects. Higher numbers trade off more memory use for — in some situations — improved efficiency.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

tcmalloc_pageheap_free_bytes

Free Heap Memory   Number of bytes in free, mapped pages in page heap. These bytes can be used to fulfill allocation requests. They always count towards virtual memory usage, and unless the underlying memory is swapped out by the OS, they also count towards physical memory usage.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

tcmalloc_pageheap_unmapped_bytes

Unmapped Heap Memory   Number of bytes in free, unmapped pages in page heap. These are bytes that have been released back to the OS, possibly by one of the MallocExtension "Release" calls. They can be used to fulfill allocation requests, but typically incur a page fault. They always count towards virtual memory usage, and depending on the OS, typically do not count towards physical memory usage.

Entity Type

server

Unit

bytes

Type

gauge

Level

info

 

threads_running

Threads Running   Current number of running threads

Entity Type

server

Unit

threads

Type

gauge

Level

info

 

undo_delta_block_gc_delete_duration

Undo Delta Block GC Delete Duration   Time spent deleting ancient UNDO delta blocks.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

undo_delta_block_gc_init_duration

Undo Delta Block GC Init Duration   Time spent initializing ancient UNDO delta blocks.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

undo_delta_block_gc_perform_duration

Undo Delta Block GC Perform Duration   Time spent running the maintenance operation to GC ancient UNDO delta blocks.

Entity Type

tablet

Unit

milliseconds

Type

histogram

Level

info

 

upserts_as_updates

Upserts converted into updates   Number of upserts which were applied as updates because the key already existed.

Entity Type

tablet

Unit

rows

Type

counter

Level

info

 

uptime

Server Uptime   Time interval since the server has started

Entity Type

server

Unit

microseconds

Type

gauge

Level

info

 

voluntary_context_switches

Voluntary Context Switches   Total voluntary context switches

Entity Type

server

Unit

context switches

Type

counter

Level

info

 

Debug Metrics

Metrics tagged as 'debug' are diagnostically helpful but generally not monitored during normal operation.

all_transactions_inflight

Ops In Flight   Number of ops currently in-flight, including any type.

Entity Type

tablet

Unit

transactions

Type

gauge

Level

debug

 

alter_schema_transactions_inflight

Alter Schema Ops In Flight   Number of alter schema ops currently in-flight

Entity Type

tablet

Unit

transactions

Type

gauge

Level

debug

 

block_cache_evictions

Block Cache Evictions   Number of blocks evicted from the cache

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_hits

Block Cache Hits   Number of lookups that found a block

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_hits_caching

Block Cache Hits (Caching)   Number of lookups that were expecting a block that found one.Use this number instead of cache_hits when trying to determine how efficient the cache is

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_inserts

Block Cache Inserts   Number of blocks inserted in the cache

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_lookups

Block Cache Lookups   Number of blocks looked up from the cache

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_misses

Block Cache Misses   Number of lookups that didn’t yield a block

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_cache_misses_caching

Block Cache Misses (Caching)   Number of lookups that were expecting a block that didn’t yield one.Use this number instead of cache_misses when trying to determine how efficient the cache is

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_blocks_created

Data Blocks Created   Number of data blocks that were created since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_blocks_deleted

Data Blocks Deleted   Number of data blocks that were deleted since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_bytes_read

Block Data Bytes Read   Number of bytes of block data read since service start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

block_manager_total_bytes_written

Block Data Bytes Written   Number of bytes of block data written since service start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

block_manager_total_disk_sync

Block Data Disk Synchronization Count   Number of disk synchronizations of block data since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_readable_blocks

Data Blocks Opened For Read   Number of data blocks opened for reading since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

block_manager_total_writable_blocks

Data Blocks Opened For Write   Number of data blocks opened for writing since service start

Entity Type

server

Unit

blocks

Type

counter

Level

debug

 

bloom_lookups

Bloom Filter Lookups   Number of times a bloom filter was consulted

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

bloom_lookups_per_op

Bloom Lookups per Operation   Tracks the number of bloom filter lookups performed by each operation. A single operation may perform several bloom filter lookups if the tablet is not fully compacted. High frequency of high values may indicate that compaction is falling behind.

Entity Type

tablet

Unit

probes

Type

histogram

Level

debug

 

builtin_ntp_max_errors

Built-In NTP Maximum Time Errors   Statistics on the maximum true time error computed by built-in NTP client

Entity Type

server

Unit

microseconds

Type

histogram

Level

debug

 

builtin_ntp_time

Built-in NTP Time   Latest true time as tracked by built-in NTP client

Entity Type

server

Unit

microseconds

Type

gauge

Level

debug

 

bytes_flushed

Bytes Flushed   Amount of data that has been flushed to disk by this tablet.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

code_cache_hits

Codegen Cache Hits   Number of codegen cache hits since start

Entity Type

server

Unit

hits

Type

counter

Level

debug

 

code_cache_queries

Codegen Cache Queries   Number of codegen cache queries (hits + misses) since start

Entity Type

server

Unit

queries

Type

counter

Level

debug

 

commit_wait_duration

Commit-Wait Duration   Time spent waiting for COMMIT_WAIT external consistency writes for this tablet.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

compact_rs_running

RowSet Compactions Running   Number of RowSet compactions currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

delete_ignore_errors

Delete Ignore Errors   Number of delete ignore operations for this tablet which were ignored due to an error since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

deleted_rowset_estimated_retained_bytes

Estimated Deletable Bytes Retained in Deleted Rowsets   Estimated bytes of deletable data in deleted rowsets for this tablet.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

debug

 

deleted_rowset_gc_bytes_deleted

Deleted Rowsets GC Bytes Deleted   Number of bytes deleted by garbage-collecting deleted rowsets.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

deleted_rowset_gc_running

Deleted Rowset GC Running   Number of deleted rowset GC operations currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

delta_file_lookups

Delta File Lookups   Number of times a delta file was consulted

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

delta_major_compact_rs_running

Major Delta Compactions Running   Number of delta major compactions currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

delta_minor_compact_rs_running

Minor Delta Compactions Running   Number of delta minor compactions currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

file_cache_evictions

File Cache Evictions   Number of file descriptors evicted from the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_hits

File Cache Hits   Number of lookups that found a file descriptor

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_hits_caching

File Cache Hits (Caching)   Number of lookups that were expecting a file descriptor that found one. Use this number instead of cache_hits when trying to determine how efficient the cache is

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_inserts

File Cache Inserts   Number of file descriptors inserted in the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_lookups

File Cache Lookups   Number of file descriptors looked up from the cache

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_misses

File Cache Misses   Number of lookups that didn’t yield a file descriptor

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

file_cache_misses_caching

File Cache Misses (Caching)   Number of lookups that were expecting a file descriptor that didn’t yield one. Use this number instead of cache_misses when trying to determine how efficient the cache is

Entity Type

server

Unit

entries

Type

counter

Level

debug

 

flush_dms_running

DeltaMemStore Flushes Running   Number of delta memstore flushes currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

flush_mrs_running

MemRowSet Flushes Running   Number of MemRowSet flushes currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

glog_info_messages

INFO-level Log Messages   Number of INFO-level log messages emitted by the application.

Entity Type

server

Unit

messages

Type

counter

Level

debug

 

hybrid_clock_max_errors

Hybrid Clock Maximum Errors   The statistics on the maximum error of the underlying clock

Entity Type

server

Unit

microseconds

Type

histogram

Level

debug

 

in_progress_ops

Operations in Progress   Number of operations in the peer’s queue ack’d by a minority of peers.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

insert_ignore_errors

Insert Ignore Errors   Number of insert ignore operations for this tablet which were ignored due to an error since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

insertions_failed_dup_key

Duplicate Key Inserts   Number of inserts which failed because the key already existed

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

key_file_lookups

Key File Lookups   Number of times a key cfile was consulted

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

key_file_lookups_per_op

Key Lookups per Operation   Tracks the number of key file lookups performed by each operation. A single operation may perform several key file lookups if the tablet is not fully compacted and if bloom filters are not effectively culling lookups.

Entity Type

tablet

Unit

probes

Type

histogram

Level

debug

 

last_read_elapsed_time

Seconds Since Last Read   The elapsed time, in seconds, since the last read operation on this tablet, or since this Tablet object was created on current tserver if it hasn’t been read since then.

Entity Type

tablet

Unit

seconds

Type

gauge

Level

debug

 

last_write_elapsed_time

Seconds Since Last Write   The elapsed time, in seconds, since the last write operation on this tablet, or since this Tablet object was created on current tserver if it hasn’t been written to since then.

Entity Type

tablet

Unit

seconds

Type

gauge

Level

debug

 

log_block_manager_dead_containers_deleted

Number of Dead Block Containers Deleted   Number of full (but dead) block containers that were deleted

Entity Type

server

Unit

log block containers

Type

counter

Level

debug

 

log_block_manager_holes_punched

Number of Holes Punched   Number of holes punched since service start

Entity Type

server

Unit

holes

Type

counter

Level

debug

 

log_bytes_logged

Bytes Written to WAL   Number of bytes logged since service start

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

log_cache_num_ops

Log Cache Operation Count   Number of operations in the log cache.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

log_cache_size

Log Cache Memory Usage   Amount of memory in use for caching the local log.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

debug

 

log_entry_batches_per_group

Log Group Commit Batch Size   Number of log entry batches in a group commit group

Entity Type

tablet

Unit

requests

Type

histogram

Level

debug

 

log_group_commit_latency

Log Group Commit Latency   Microseconds spent on committing an entire group

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

log_reader_bytes_read

Bytes Read From Log   Data read from the WAL since tablet start

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

log_reader_entries_read

Entries Read From Log   Number of entries read from the WAL since tablet start

Entity Type

tablet

Unit

entries

Type

counter

Level

debug

 

log_roll_latency

Log Roll Latency   Microseconds spent on rolling over to a new log segment file

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

majority_done_ops

Leader Operations Acked by Majority   Number of operations in the leader queue ack’d by a majority but not all peers. This metric is always zero for followers.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

mrs_lookups

MemRowSet Lookups   Number of times a MemRowSet was consulted.

Entity Type

tablet

Unit

probes

Type

counter

Level

debug

 

raft_term

Current Raft Consensus Term   Current Term of the Raft Consensus algorithm. This number increments each time a leader election is started.

Entity Type

tablet

Unit

units

Type

gauge

Level

debug

 

scanner_bytes_returned

Scanner Bytes Returned   Number of bytes returned by scanners to clients. This count is measured after predicates are applied and the data is decoded for consumption by clients, and thus is not a reflection of the amount of work being done by scanners.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

scanner_bytes_scanned_from_disk

Scanner Bytes Scanned From Disk   Number of bytes read by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual IO that has been caused by scan operations compared to the Scanner Bytes Returned metric. Note that this only counts data that has been flushed to disk, and does not include data read from in-memory stores. However, itincludes both cache misses and cache hits.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

scanner_cells_returned

Scanner Cells Returned   Number of table cells returned by scanners to clients. This count is measured after predicates are applied, and thus is not a reflection of the amount of work being done by scanners.

Entity Type

tablet

Unit

cells

Type

counter

Level

debug

 

scanner_cells_scanned_from_disk

Scanner Cells Scanned From Disk   Number of table cells processed by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual table cells that have been processed by scan operations compared to the Scanner Cells Returned metric. Note that this only counts data that has been flushed to disk, and does not include data read from in-memory stores. However, itincludes both cache misses and cache hits.

Entity Type

tablet

Unit

cells

Type

counter

Level

debug

 

scanner_rows_returned

Scanner Rows Returned   Number of rows returned by scanners to clients. This count is measured after predicates are applied, and thus is not a reflection of the amount of work being done by scanners.

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

scanner_rows_scanned

Scanner Rows Scanned   Number of rows processed by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual table rows that have been processed by scan operations compared to the Scanner Rows Returned metric.

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

scans_started

Scans Started   Number of scanners which have been started on this tablet

Entity Type

tablet

Unit

scanners

Type

counter

Level

debug

 

snapshot_read_inflight_wait_duration

Time Waiting For Snapshot Reads   Time spent waiting for in-flight writes to complete for READ_AT_SNAPSHOT scans.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

tablet_active_scanners

Active Scanners   Number of scanners that are currently active on this tablet

Entity Type

tablet

Unit

scanners

Type

gauge

Level

debug

 

tablet_copy_bytes_fetched

Bytes Fetched By Tablet Copy   Number of bytes fetched during tablet copy operations since server start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

tablet_copy_bytes_sent

Bytes Sent For Tablet Copy   Number of bytes sent during tablet copy operations since server start

Entity Type

server

Unit

bytes

Type

counter

Level

debug

 

threads_started

Threads Started   Total number of threads started on this server

Entity Type

server

Unit

threads

Type

counter

Level

debug

 

time_since_last_leader_heartbeat

Time Since Last Leader Heartbeat   The time elapsed since the last heartbeat from the leader in milliseconds. This metric is identically zero on a leader replica.

Entity Type

tablet

Unit

milliseconds

Type

gauge

Level

debug

 

undo_delta_block_estimated_retained_bytes

Estimated Deletable Bytes Retained in Undo Delta Blocks   Estimated bytes of deletable data in undo delta blocks for this tablet. May be an overestimate.

Entity Type

tablet

Unit

bytes

Type

gauge

Level

debug

 

undo_delta_block_gc_bytes_deleted

Undo Delta Block GC Bytes Deleted   Number of bytes deleted by garbage-collecting old UNDO delta blocks on this tablet since this server was restarted. Does not include bytes garbage collected during compactions.

Entity Type

tablet

Unit

bytes

Type

counter

Level

debug

 

undo_delta_block_gc_running

Undo Delta Block GC Running   Number of UNDO delta block GC operations currently running.

Entity Type

tablet

Unit

operations

Type

gauge

Level

debug

 

update_ignore_errors

Update Ignore Errors   Number of update ignore operations for this tablet which were ignored due to an error since service start

Entity Type

tablet

Unit

rows

Type

counter

Level

debug

 

write_op_duration_client_propagated_consistency

Write Op Duration with Propagated Consistency   Duration of writes to this tablet with external consistency set to CLIENT_PROPAGATED.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

write_op_duration_commit_wait_consistency

Write Op Duration with Commit-Wait Consistency   Duration of writes to this tablet with external consistency set to COMMIT_WAIT.

Entity Type

tablet

Unit

microseconds

Type

histogram

Level

debug

 

write_transactions_inflight

Write Ops In Flight   Number of write ops currently in-flight

Entity Type

tablet

Unit

transactions

Type

gauge

Level

debug