Skip to content

Latest commit

 

History

History
1228 lines (1097 loc) · 76.1 KB

stats.org

File metadata and controls

1228 lines (1097 loc) · 76.1 KB

EP Stats

1 Getting Started

For introductory information on stats within membase, start with the membase wiki stats page.

2 Stats Definitions

2.1 Toplevel Stats

StatDescription
uuidThe unique identifier for the bucket
ep_versionVersion number of ep_engine
ep_storage_ageSeconds since most recently
stored object was initially queued
ep_storage_age_highwatep_storage_age high water mark
ep_startup_timeSystem-generated engine startup time
ep_data_ageSeconds since most recently
stored object was modified
ep_data_age_highwatep_data_age high water mark
ep_too_youngNumber of times an object was
not stored due to being too young
ep_too_oldNumber of times an object was
stored after being dirty too long
ep_num_workersGlobal number of shared worker threads
ep_bucket_priorityPriority assigned to the bucket
ep_total_enqueuedTotal number of items queued for
persistence
ep_total_new_itemsTotal number of persisted new items
ep_total_del_itemsTotal number of persisted deletions
ep_total_persistedTotal number of items persisted
ep_item_flush_failedNumber of times an item failed to
flush due to storage errors
ep_item_commit_failedNumber of times a transaction failed
to commit due to storage errors
ep_item_begin_failedNumber of times a transaction failed
to start due to storage errors
ep_expired_accessNumber of times an item was expired on
application access.
ep_expired_pagerNumber of times an item was expired by
ep engine item pager
ep_item_flush_expiredNumber of times an item is not flushed
due to the expiry of the item
ep_queue_sizeNumber of items queued for storage
ep_flusher_todoNumber of items currently being
written
ep_flusher_stateCurrent state of the flusher thread
ep_commit_numTotal number of write commits
ep_commit_timeNumber of milliseconds of most recent
commit
ep_commit_time_totalCumulative milliseconds spent
committing
ep_vbucket_delNumber of vbucket deletion events
ep_vbucket_del_failNumber of failed vbucket deletion
events
ep_vbucket_del_max_walltimeMax wall time (µs) spent by deleting
a vbucket
ep_vbucket_del_avg_walltimeAvg wall time (µs) spent by deleting
a vbucket
ep_pending_compactionsNumber of pending vbucket compactions
ep_rollback_countNumber of rollbacks on consumer
ep_flush_duration_totalCumulative seconds spent flushing
ep_flush_allTrue if disk flush_all is scheduled
ep_num_ops_get_metaNumber of getMeta operations
ep_num_ops_set_metaNumber of setWithMeta operations
ep_num_ops_del_metaNumber of delWithMeta operations
ep_num_ops_set_meta_res_failedNumber of setWithMeta ops that failed
conflict resolution
ep_num_ops_del_meta_res_failedNumber of delWithMeta ops that failed
conflict resolution
ep_num_ops_set_ret_metaNumber of setRetMeta operations
ep_num_ops_del_ret_metaNumber of delRetMeta operations
ep_num_ops_get_meta_on_set_metaNum of background getMeta operations
spawn due to setWithMeta operations
curr_itemsNum items in active vbuckets (temp +
live)
curr_temp_itemsNum temp items in active vbuckets
curr_items_totNum current items including those not
active (replica, dead and pending
states)
ep_kv_sizeMemory used to store item metadata,
keys and values, no matter the
vbucket’s state. If an item’s value is
ejected, this stats will be
decremented by the size of the item’s
value.
ep_blob_numThe number of blob objects in the cache
ep_blob_overheadThe “unused” memory caused by the
allocator returning bigger chunks than
requested
ep_value_sizeMemory used to store values for
resident keys
ep_storedval_sizeMemory used by storedval objects
ep_storedval_overheadThe “unused” memory caused by the
allocator returning bigger chunks than
requested
ep_storedval_numThe number of storedval objects
allocated
ep_overheadExtra memory used by transient data
like persistence queues, replication
queues, checkpoints, etc
ep_item_numThe number of item objects allocated
ep_mem_low_watLow water mark for auto-evictions
ep_mem_high_watHigh water mark for auto-evictions
ep_total_cache_sizeThe total byte size of all items, no
matter the vbucket’s state, no matter
if an item’s value is ejected
ep_oom_errorsNumber of times unrecoverable OOMs
happened while processing operations
ep_tmp_oom_errorsNumber of times temporary OOMs
happened while processing operations
ep_mem_tracker_enabledTrue if memory usage tracker is
enabled
ep_bg_fetchedNumber of items fetched from disk
ep_bg_meta_fetchedNumber of meta items fetched from disk
ep_bg_remaining_jobsNumber of remaining bg fetch jobs
ep_max_bg_remaining_jobsMax number of remaining bg fetch jobs
that we have seen in the queue so far
ep_tap_bg_fetchedNumber of tap disk fetches
ep_tap_bg_fetch_requeuedNumber of times a tap bg fetch task is
requeued
ep_num_pager_runsNumber of times we ran pager loops
to seek additional memory
ep_num_expiry_pager_runsNumber of times we ran expiry pager
loops to purge expired items from
memory/disk
ep_num_access_scanner_runsNumber of times we ran accesss scanner
to snapshot working set
ep_access_scanner_num_itemsNumber of items that last access
scanner task swept to access log.
ep_access_scanner_task_timeTime of the next access scanner task
(GMT), NOT_SCHEDULED if access scanner
has been disabled
ep_access_scanner_last_runtimeNumber of seconds that last access
scanner task took to complete.
ep_items_rm_from_checkpointsNumber of items removed from closed
unreferenced checkpoints
ep_num_value_ejectsNumber of times item values got
ejected from memory to disk
ep_num_eject_failuresNumber of items that could not be
ejected
ep_num_not_my_vbucketsNumber of times Not My VBucket
exception happened during runtime
ep_tap_keepaliveTap keepalive time
ep_dbnameDB path
ep_io_num_readNumber of io read operations
ep_io_num_writeNumber of io write operations
ep_io_read_bytesNumber of bytes read (key + values)
ep_io_write_bytesNumber of bytes written (key + values)
ep_pending_opsNumber of ops awaiting pending
vbuckets
ep_pending_ops_totalTotal blocked pending ops since reset
ep_pending_ops_maxMax ops seen awaiting 1 pending
vbucket
ep_pending_ops_max_durationMax time (µs) used waiting on pending
vbuckets
ep_bg_num_samplesThe number of samples included in the
avgerage
ep_bg_min_waitThe shortest time (µs) in the wait
queue
ep_bg_max_waitThe longest time (µs) in the wait
queue
ep_bg_wait_avgThe average wait time (µs) for an item
before it’s serviced by the dispatcher
ep_bg_min_loadThe shortest load time (µs)
ep_bg_max_loadThe longest load time (µs)
ep_bg_load_avgThe average time (µs) for an item to
be loaded from the persistence layer
ep_num_non_residentThe number of non-resident items
ep_bg_waitThe total elapse time for the wait
queue
ep_bg_loadThe total elapse time for items to be
loaded from the persistence layer
ep_allow_data_loss_during_shutdownWhether data loss is allowed during
server shutdown
ep_alog_block_sizeAccess log block size
ep_alog_pathPath to the access log
ep_access_scanner_enabledStatus of access scanner task
ep_alog_sleep_timeInterval between access scanner runs
in minutes
ep_alog_task_timeHour in GMT time when access scanner
task is scheduled to run
ep_backendThe backend that is being used for
data persistence
ep_backfill_mem_thresholdThe maximum percentage of memory that
the backfill task can consume before
it is made to back off.
ep_bg_fetch_delayThe amount of time to wait before
doing a background fetch
ep_chk_max_itemsThe number of items allowed in a
checkpoint before a new one is created
ep_chk_periodThe maximum lifetime of a checkpoint
before a new one is created
ep_chk_persistence_remainsNumber of remaining vbuckets for
checkpoint persistence
ep_chk_persistence_timeoutTimeout for vbucket checkpoint
persistence
ep_chk_remover_stimeThe time interval for purging closed
checkpoints from memory
ep_config_fileThe location of the ep-engine config
file
ep_couch_bucketThe name of this bucket
ep_couch_hostThe hostname that the couchdb views
server is listening on
ep_couch_portThe port the couchdb views server is
listening on
ep_couch_reconnect_sleeptimeThe amount of time to wait before
reconnecting to couchdb
ep_data_traffic_enabledWhether or not data traffic is enabled
for this bucket
ep_db_data_sizeTotal size of valid data in db files
ep_db_file_sizeTotal size of the db files
ep_degraded_modeTrue if the engine is either warming
up or data traffic is disabled
ep_enable_chk_mergeTrue if merging closed checkpoints is
enabled.
ep_exp_pager_stimeThe time interval for purging expired
items from memory
ep_failpartialwarmupTrue if we want kill the bucket if
warmup fails
ep_flushall_enabledTrue if this bucket allows the use of
the flush_all command
ep_getl_default_timeoutThe default getl lock duration
ep_getl_max_timeoutThe maximum getl lock duration
ep_ht_locksThe amount of locks per vb hashtable
ep_ht_sizeThe initial size of each vb hashtable
ep_item_num_based_new_chkTrue if the number of items in the
current checkpoint plays a role in a
new checkpoint creation
ep_keep_closed_chksTrue if we want to keep the closed
checkpoints for each vbucket unless
the memory usage is above high water
mark
ep_max_checkpointsThe maximum amount of checkpoints that
can be in memory per vbucket
ep_max_item_sizeThe maximum value size
ep_max_sizeThe maximum amount of memory this
bucket can use
ep_max_vbucketsThe maximum amount of vbuckets that
can exist in this bucket
ep_mutation_mem_thresholdThe ratio of total memory available
that we should start sending temp oom
or oom message when hitting
ep_pager_active_vb_pcntActive vbuckets paging percentage
ep_tap_ack_grace_periodThe amount of time to wait for a tap
acks before disconnecting
ep_tap_ack_initial_sequence_numberThe initial sequence number for a tap
ack when a tap stream is created
ep_tap_ack_intervalThe amount of messages a tap producer
should send before requesting an ack
ep_tap_ack_window_sizeThe maximum amount of ack requests
that can be sent before the consumer
sends a response ack. When the window
is full the tap stream is paused.
ep_tap_backfill_residentThe resident ratio for deciding how to
do backfill. If under the ratio we
schedule full disk backfill. If above
the ratio then we do bg fetches for
non-resident items.
ep_tap_backlog_limitThe maximum amount of backfill items
that can be in memory waiting to be
sent to the tap consumer
ep_tap_backoff_periodThe number of seconds the tap
connection
ep_tap_bg_fetch_requeuedNumber of times a tap bg fetch task is
requeued
ep_tap_bg_max_pendingThe maximum number of bg jobs a tap
connection may have
ep_tap_noop_intervalNumber of seconds between a noop is
sent on an idle connection
ep_tap_requeue_sleep_timeThe amount of time to wait before a
failed tap item is requeued
ep_tap_throttle_cap_pcntPercentage of total items in write
queue at which we throttle tap input
ep_tap_throttle_queue_capMax size of a write queue to throttle
incoming tap input
ep_tap_throttle_thresholdPercentage of max mem at which we
begin NAKing tap input
ep_uncommitted_itemsThe amount of items that have not been
written to disk
ep_warmupShows if warmup is enabled / disabled
ep_warmup_batch_sizeThe size of each batch loaded during
warmup
ep_warmup_dupsNumber of Duplicate items encountered
during warmup
ep_warmup_min_items_thresholdPercentage of total items warmed up
before we enable traffic
ep_warmup_min_memory_thresholdPercentage of max mem warmed up before
we enable traffic
ep_warmup_oomThe amount of oom errors that occured
during warmup
ep_warmup_threadThe status of the warmup thread
ep_warmup_timeThe amount of time warmup took
ep_workload_patternWorkload pattern (mixed, read_heavy,
write_heavy) monitored at runtime

2.2 vBucket total stats

StatDescription
ep_vb_totalTotal vBuckets (count)
curr_items_totTotal number of items
curr_itemsNumber of active items in memory
curr_temp_itemsNumber of temporary items in memory
vb_dead_numNumber of dead vBuckets
ep_diskqueue_itemsTotal items in disk queue
ep_diskqueue_memoryTotal memory used in disk queue
ep_diskqueue_fillTotal enqueued items on disk queue
ep_diskqueue_drainTotal drained items on disk queue
ep_diskqueue_pendingTotal bytes of pending writes
ep_vb_snapshot_totalTotal VB state snapshots persisted in disk
ep_meta_data_memoryTotal memory used by meta data
ep_meta_data_diskTotal disk used by meta data

2.2.1 Active vBucket class stats

StatDescription
vb_active_numNumber of active vBuckets
vb_active_curr_itemsNumber of in memory items
vb_active_num_non_residentNumber of non-resident items
vb_active_perc_mem_resident% memory resident
vb_active_ejectNumber of times item values got ejected
vb_active_expiredNumber of times an item was expired
vb_active_ht_memoryMemory overhead of the hashtable
vb_active_itm_memoryTotal item memory
vb_active_meta_data_memoryTotal metadata memory
vb_active_meta_data_diskTotal metadata disk
vb_active_ops_createNumber of create operations
vb_active_ops_updateNumber of update operations
vb_active_ops_deleteNumber of delete operations
vb_active_ops_rejectNumber of rejected operations
vb_active_queue_sizeActive items in disk queue
vb_active_queue_memoryMemory used for disk queue
vb_active_queue_ageSum of disk queue item age in milliseconds
vb_active_queue_pendingTotal bytes of pending writes
vb_active_queue_fillTotal enqueued items
vb_active_queue_drainTotal drained items

2.2.2 Replica vBucket stats

StatDescription
vb_replica_numNumber of replica vBuckets
vb_replica_curr_itemsNumber of in memory items
vb_replica_num_non_residentNumber of non-resident items
vb_replica_perc_mem_resident% memory resident
vb_replica_ejectNumber of times item values got ejected
vb_replica_expiredNumber of times an item was expired
vb_replica_ht_memoryMemory overhead of the hashtable
vb_replica_itm_memoryTotal item memory
vb_replica_meta_data_memoryTotal metadata memory
vb_replica_meta_data_diskTotal metadata disk
vb_replica_ops_createNumber of create operations
vb_replica_ops_updateNumber of update operations
vb_replica_ops_deleteNumber of delete operations
vb_replica_ops_rejectNumber of rejected operations
vb_replica_queue_sizeReplica items in disk queue
vb_replica_queue_memoryMemory used for disk queue
vb_replica_queue_ageSum of disk queue item age in milliseconds
vb_replica_queue_pendingTotal bytes of pending writes
vb_replica_queue_fillTotal enqueued items
vb_replica_queue_drainTotal drained items

2.2.3 Pending vBucket stats

StatDescription
vb_pending_numNumber of pending vBuckets
vb_pending_curr_itemsNumber of in memory items
vb_pending_num_non_residentNumber of non-resident items
vb_pending_perc_mem_resident% memory resident
vb_pending_ejectNumber of times item values got ejected
vb_pending_expiredNumber of times an item was expired
vb_pending_ht_memoryMemory overhead of the hashtable
vb_pending_itm_memoryTotal item memory
vb_pending_meta_data_memoryTotal metadata memory
vb_pending_meta_data_diskTotal metadata disk
vb_pending_ops_createNumber of create operations
vb_pending_ops_updateNumber of update operations
vb_pending_ops_deleteNumber of delete operations
vb_pending_ops_rejectNumber of rejected operations
vb_pending_queue_sizePending items in disk queue
vb_pending_queue_memoryMemory used for disk queue
vb_pending_queue_ageSum of disk queue item age in milliseconds
vb_pending_queue_pendingTotal bytes of pending writes
vb_pending_queue_fillTotal enqueued items
vb_pending_queue_drainTotal drained items

2.3 vBucket detail stats

The stats below are listed for each vbucket.

StatDescription
num_itemsNumber of items in this vbucket
num_tmp_itemsNumber of temporary items in memory
num_non_residentNumber of non-resident items
vb_pending_perc_mem_resident% memory resident
vb_pending_ejectNumber of times item values got ejected
vb_pending_expiredNumber of times an item was expired
ht_memoryMemory overhead of the hashtable
ht_item_memoryTotal item memory
ht_cache_sizeTotal size of cache (Includes non resident
items)
num_ejectsNumber of times an item was ejected from
memory
ops_createNumber of create operations
ops_updateNumber of update operations
ops_deleteNumber of delete operations
ops_rejectNumber of rejected operations
queue_sizePending items in disk queue
queue_memoryMemory used for disk queue
queue_ageSum of disk queue item age in milliseconds
queue_fillTotal enqueued items
queue_drainTotal drained items
pending writesTotal bytes of pending writes
db_data_sizeTotal size of valid data on disk
db_file_sizeTotal size of the db file
high_seqnoThe last seqno assigned by this vbucket
purge_seqnoThe last seqno purged by the compactor
uuidThe current vbucket uuid

2.4 vBucket seqno stats

StatsDescription
——————————+——————————————–
high_seqnoThe last seqno assigned by this vbucket
purge_seqnoThe last seqno purged by the compactor
uuidThe current vbucket uuid

2.5 Tap stats

ep_tap_ack_grace_periodThe amount of time to wait for a tap acks
before disconnecting
ep_tap_ack_intervalThe amount of messages a tap producer
should send before requesting an ack
ep_tap_ack_window_sizeThe maximum amount of ack requests that
can be sent before the consumer sends a
response ack. When the window is full the
tap stream is paused
ep_tap_queue_backfillremainingNumber of items needing to be backfilled
ep_tap_total_backlog_sizeNumber of remaining items for replication
ep_tap_total_queueSum of tap queue sizes on the current
tap queues
ep_tap_total_fetchedSum of all tap messages sent
ep_tap_bg_max_pendingThe maximum number of bg jobs a tap
connection may have
ep_tap_bg_fetchedNumber of tap disk fetches
ep_tap_bg_fetch_requeuedNumber of times a tap bg fetch task is
requeued
ep_tap_fg_fetchedNumber of tap memory fetches
ep_tap_deletesNumber of tap deletion messages sent
ep_tap_throttledNumber of tap messages refused due to
throttling
ep_tap_countNumber of tap connections
ep_tap_bg_num_samplesThe number of tap bg fetch samples
included in the avg
ep_tap_bg_min_waitThe shortest time (µs) for a tap item
before it is serviced by the dispatcher
ep_tap_bg_max_waitThe longest time (µs) for a tap item
before it is serviced by the dispatcher
ep_tap_bg_wait_avgThe average wait time (µs) for a tap item
before it is serviced by the dispatcher
ep_tap_bg_min_loadThe shortest time (µs) for a tap item to
be loaded from the persistence layer
ep_tap_bg_max_loadThe longest time (µs) for a tap item to
be loaded from the persistence layer
ep_tap_bg_load_avgThe average time (µs) for a tap item to
be loaded from the persistence layer
ep_tap_noop_intervalThe number of secs between a noop is
added to an idle connection
ep_tap_backoff_periodThe number of seconds the tap connection
should back off after receiving ETMPFAIL
ep_tap_queue_fillTotal enqueued items
ep_tap_queue_drainTotal drained items
ep_tap_queue_backoffTotal back-off items
ep_tap_queue_backfillNumber of backfill remaining
ep_tap_queue_itemondiskNumber of items remaining on disk
ep_tap_throttle_thresholdPercentage of memory in use before we
throttle tap streams
ep_tap_throttle_queue_capDisk write queue cap to throttle
tap streams

2.5.1 Per Tap Client Stats

Each stat begins with ep_tapq: followed by a unique client_id and another colon. For example, if your client is named, slave1, the qlen stat would be ep_tapq:slave1:qlen.

typeThe kind of tap connection (producer orPC
consumer)
createdCreation time for the tap connectionPC
supports_acktrue if the connection use acksPC
connectedtrue if this client is connectedPC
disconnectsNumber of disconnects from this clientPC
reservedtrue if the tap stream is reservedP
suspendedtrue if the tap stream is suspendedP
qlenQueue size for the given client_idP
qlen_high_priHigh priority tap queue itemsP
qlen_low_priLow priority tap queue itemsP
vb_filtersSize of connection vbucket filter setP
vb_filterThe content of the vbucket filterP
rec_fetchedTap messages sent to the clientP
rec_skippedNumber of messages skipped due toP
tap reconnect with a different filterP
idleTrue if this connection is idleP
has_queued_itemTrue if there are any remaining itemsP
from hash table or disk
bg_result_sizeNumber of ready background resultsP
bg_jobs_issuedNumber of background jobs startedP
bg_jobs_completedNumber of background jobs completedP
flagsConnection flags set by the clientP
pending_disconnecttrue if we’re hanging up on this clientP
pausedtrue if this client is blockedP
pending_backfilltrue if we’re still backfilling keysP
for this connectionP
pending_disk_backfilltrue if we’re still backfilling keysP
from disk for this connectionP
backfill_completedtrue if all items from backfill isP
successfully transmitted to the clientP
backfill_start_timestampTimestamp of backfill startP
reconnectsNumber of reconnects from this clientP
backfill_ageThe age of the start of the backfillP
ack_seqnoThe current tap ACK sequence numberP
recv_ack_seqnoLast receive tap ACK sequence numberP
ack_log_sizeTap ACK backlog sizeP
ack_window_fulltrue if our tap ACK window is fullP
seqno_ack_requestedThe seqno of the ack message that theP
producer is wants to get a response for
expiresWhen this ACK backlog expiresP
queue_memoryMemory used for tap queueP
queue_fillTotal queued itemsP
queue_drainTotal drained itemsP
queue_backoffTotal back-off itemsP
queue_backfillremainingNumber of backfill remainingP
queue_itemondiskNumber of items remaining on diskP
total_backlog_sizeNum of remaining items for replicationP
total_noopsNumber of NOOP messages sentP
num_checkpoint_endNumber of chkpoint end operationsC
num_checkpoint_end_failedNumber of chkpoint end operations failedC
num_checkpoint_startNumber of chkpoint end operationsC
num_checkpoint_start_failedNumber of chkpoint end operations failedC
num_deleteNumber of delete operationsC
num_delete_failedNumber of failed delete operationsC
num_flushNumber of flush operationsC
num_flush_failedNumber of failed flush operationsC
num_mutationNumber of mutation operationsC
num_mutation_failedNumber of failed mutation operationsC
num_opaqueNumber of opaque operationC
num_opaque_failedNumber of failed opaque operationsC
num_vbucket_setNumber of vbucket set operationsC
num_vbucket_set_failedNumber of failed vbucket set operationsC
num_unknownNumber of unknown operationsC

2.6 Tap Aggregated Stats

Aggregated tap stats allow named tap connections to be logically grouped and aggregated together by prefixes.

For example, if all of your tap connections started with rebalance_ or replication_, you could call stats tapagg _ to request stats grouped by everything before the first _ character, giving you a set for rebalance and a set for replication.

2.6.1 Results

[prefix]:countNumber of connections matching this prefix
[prefix]:qlenTotal length of queues with this prefix
[prefix]:backfill_remainingNumber of items needing to be backfilled
[prefix]:backoffTotal number of backoff events
[prefix]:drainTotal number of items drained
[prefix]:fillTotal number of items filled
[prefix]:itemondiskNumber of items remaining on disk
[prefix]:total_backlog_sizeNum of remaining items for replication

2.7 Dcp Stats

Each stat begins with ep_dcpq: followed by a unique client_id and another colon. For example, if your client is named, slave1, the created stat would be ep_dcpq:slave1:created.

***Consumer Connections

connectedTrue if this client is connected
createdCreation time for the tap connection
pending_disconnectTrue if we’re hanging up on this client
reservedTrue if the dcp stream is reserved
supports_ackTrue if the connection use flow control
total_acked_bytesThe amount of bytes that the consumer has acked
typeThe connection type (producer, consumer, or notifier)

****Per Stream Stats

buffer_bytesThe amount of unprocessed bytes
buffer_itemsThe amount of unprocessed items
end_seqnoThe seqno where this stream should end
flagsThe flags used to create this stream
items_readyWhether the stream has messages ready to send
opaqueThe unique stream identifier
snap_end_seqnoThe start seqno of the last snapshot received
snap_start_seqnoThe end seqno of the last snapshot received
start_seqnoThe start start seqno used to create this stream
stateThe stream state (pending, reading, or dead)
vb_uuidThe vb uuid used to create this stream

***Producer/Notifier Connections

bytes_sentThe amount of unacked bytes sent to the consumer
connectedTrue if this client is connected
createdCreation time for the tap connection
flow_controlTrue if the connection use flow control
items_remainingThe amount of items remaining to be sent
items_sentThe amount of items already sent to the consumer
last_sent_timeThe last time this connection sent a message
max_buffer_bytesThe maximum amount of bytes that can be sent without
receiving an ack from the consumer
noop_enabledWhether or not this connection sends noops
noop_waitWhether or not this connection is waiting for a
noop response from the consumer
pending_disconnectTrue if we’re hanging up on this client
reservedTrue if the dcp stream is reserved
supports_ackTrue if the connection use flow control
total_acked_bytesThe amount of bytes that have been acked by the
consumer when flow control is enabled
total_bytes_sentThe amount of bytes already sent to the consumer
typeThe connection type (producer, consumer, or notifier)
unacked_bytesThe amount of bytes the consumer has no acked

****Per Stream Stats

backfilledThe amount of items sent from disk
end_seqnoThe seqno send mutations up to
flagsThe flags supplied in the stream request
items_readyWhether the stream has items ready to send
last_sent_seqnoThe last seqno sent by this stream
memoryThe amount of items sent from memory
opaqueThe unique stream identifier
snap_end_seqnoThe last snapshot end seqno (Used if a consumer is
resuming a stream)
snap_start_seqnoThe last snapshot start seqno (Used if a consumer is
resuming a stream)
start_seqnoThe seqno to start sending mutations from
stateThe stream state (pending, backfilling, in-memory,
takeover-send, takeover-wait, or dead)
vb_uuidThe vb uuid used in the stream request
cur_snapshot_typeThe type of the current snapshot being received
cur_snapshot_startThe start seqno of the current snapshot being received
cur_snapshot_endThe end seqno of the current snapshot being received

2.8 Dcp Aggregated Stats

Aggregated dcp stats allow dcp connections to be logically grouped and aggregated together by prefixes.

For example, if all of your dcp connections started with xdcr: or replication, you could call stats dcpagg : to request stats grouped by everything before the first : character, giving you a set for xdcr and a set for replication.

2.8.1 Results

[prefix]:countNumber of connections matching this prefix
[prefix]:producer_countTotal producer connections with this prefix
[prefix]:items_sentTotal items sent with this prefix
[prefix]:items_remainingTotal items remaining to be sent with this
prefix
[prefix]:total_bytesTotal number of bytes sent with this prefix
[prefix]:total_backlog_sizeTotal backfill items remaining to be sent
with this prefix

2.9 Timing Stats

Timing stats provide histogram data from high resolution timers over various operations within the system.

2.9.1 General Form

As this data is multi-dimensional, some parsing may be required for machine processing. It’s somewhat human readable, but the stats script mentioned in the Getting Started section above will do fancier formatting for you.

Consider the following sample stats:

STAT disk_insert_8,16 9488
STAT disk_insert_16,32 290
STAT disk_insert_32,64 73
STAT disk_insert_64,128 86
STAT disk_insert_128,256 48
STAT disk_insert_256,512 2
STAT disk_insert_512,1024 12
STAT disk_insert_1024,2048 1

This tells you that disk_insert took 8-16µs 9,488 times, 16-32µs 290 times, and so on.

The same stats displayed through the stats CLI tool would look like this:

disk_insert (10008 total)
   8us - 16us    : ( 94.80%) 9488 ###########################################
   16us - 32us   : ( 97.70%)  290 #
   32us - 64us   : ( 98.43%)   73
   64us - 128us  : ( 99.29%)   86
   128us - 256us : ( 99.77%)   48
   256us - 512us : ( 99.79%)    2
   512us - 1ms   : ( 99.91%)   12
   1ms - 2ms     : ( 99.92%)    1

2.9.2 Available Stats

The following histograms are available from “timings” in the above form to describe when time was spent doing various things:

bg_waitbg fetches waiting in the dispatcher queue
bg_loadbg fetches waiting for disk
set_with_metaset_with_meta latencies
bg_tap_waittap bg fetches waiting in the dispatcher queue
bg_tap_loadtap bg fetches waiting for disk
pending_opsclient connections blocked for operations
in pending vbuckets
storage_ageAnalogous to ep_storage_age in main stats
data_ageAnalogous to ep_data_age in main stats
get_cmdservicing get requests
arith_cmdservicing incr/decr requests
get_stats_cmdservicing get_stats requests
get_vb_cmdservicing vbucket status requests
set_vb_cmdservicing vbucket set state commands
del_vb_cmdservicing vbucket deletion commands
chk_persistence_cmdwaiting for checkpoint persistence
tap_vb_setservicing tap vbucket set state commands
tap_vb_resetservicing tap vbucket reset commands
tap_mutationservicing tap mutations
notify_iowaking blocked connections
paged_out_timetime (in seconds) objects are non-resident
disk_insertwaiting for disk to store a new item
disk_updatewaiting for disk to modify an existing item
disk_delwaiting for disk to delete an item
disk_vb_delwaiting for disk to delete a vbucket
disk_commitwaiting for a commit after a batch of updates
disk_vbstate_snapshotTime spent persisting vbucket state changes
item_alloc_sizesItem allocation size counters (in bytes)

The following histograms are available from “scheduler” and “runtimes” describing the scheduling overhead times and task runtimes incurred by various IO and Non-IO tasks respectively:

bg_fetcher_taskshistogram of scheduling overhead/task
runtimes for background fetch tasks
bg_fetcher_meta_taskshistogram of scheduling overhead/task
runtimes for background fetch meta tasks
tap_bg_fetcher_taskshistogram of scheduling overhead/task
runtimes for tap background fetch tasks
vkey_stat_bg_fetcher_taskshistogram of scheduling overhead/task
runtimes for fetching item from disk for
vkey stat tasks
warmup_taskshistogram of scheduling overhead/task
runtimes for warmup tasks
vbucket_persist_high_taskshistogram of scheduling overhead/task
runtimes for snapshot vbucket state in
high priority tasks
vbucket_deletion_taskshistogram of scheduling overhead/task
runtimes for vbucket deletion tasks
flusher_taskshistogram of scheduling overhead/task
runtimes for flusher tasks
flush_all_taskshistogram of scheduling overhead/task
runtimes for flush all tasks
compactor_taskshistogram of scheduling overhead/task
runtimes for vbucket level compaction
tasks
statsnap_taskshistogram of scheduling overhead/task
runtimes for stats snapshot tasks
mutation_log_compactor_taskshistogram of scheduling overhead/task
runtimes for access log compaction tasks
access_scanner_taskshistogram of scheduling overhead/task
runtimes for access scanner tasks
conn_notification_taskshistogram of scheduling overhead/task
runtimes for connection notification
tasks
checkpoint_remover_taskshistogram of scheduling overhead/task
runtimes for checkpoint removal tasks
vb_memory_deletion_taskshistogram of scheduling overhead/task
runtimes for memory deletion of vbucket
tasks
checkpoint_stats_taskshistogram of scheduling overhead/task
runtimes for checkpoint stats tasks
item_pager_taskshistogram of scheduling overhead/task
runtimes for item pager tasks
backfill_tasks_taskshistogram of scheduling overhead/task
runtimes for backfill tasks
tap_resume_taskshistogram of scheduling overhead/task
runtimes for resume suspended tap
connection tasks
tapconnection_reaper_taskshistogram of scheduling overhead/task
runtimes for tap/dcp connection reaper
tasks
hashtable_resize_taskshistogram of scheduling overhead/task
runtimes for hash table resizer tasks
pending_ops_taskshistogram of scheduling overhead/task
runtimes for processing dcp bufferred
items tasks
conn_manager_taskshistogram of scheduling overhead/task
runtimes for dcp/tap connection manager
tasks

2.10 Hash Stats

Hash stats provide information on your vbucket hash tables.

Requesting these stats does affect performance, so don’t do it too regularly, but it’s useful for debugging certain types of performance issues. For example, if your hash table is tuned to have too few buckets for the data load within it, the max_depth will be too large and performance will suffer.

avg_countThe average number of items per vbucket
avg_maxThe average max depth of a vbucket hash table
avg_minThe average min depth of a vbucket hash table
largest_maxThe largest hash table depth of in all vbuckets
largest_minThe the largest minimum hash table depth of all vbuckets
max_countThe largest number of items in a vbucket
min_countThe smallest number of items in a vbucket
total_countsThe total numer of items in all vbuckets

It is also possible to get more detailed hash tables stats by using ‘hash detail’. This will print per-vbucket stats.

Each stat is prefixed with vb_ followed by a number, a colon, then the individual stat name.

For example, the stat representing the size of the hash table for vbucket 0 is vb_0:size.

stateThe current state of this vbucket
sizeNumber of hash buckets
locksNumber of locks covering hash table operations
min_depthMinimum number of items found in a bucket
max_depthMaximum number of items found in a bucket
reportedNumber of items this hash table reports having
countedNumber of items found while walking the table
resizedNumber of times the hash table resized
mem_sizeRunning sum of memory used by each item
mem_size_countedCounted sum of current memory used by each item

2.11 Checkpoint Stats

Checkpoint stats provide detailed information on per-vbucket checkpoint datastructure.

Like Hash stats, requesting these stats has some impact on performance. Therefore, please do not poll them from the server frequently. Each stat is prefixed with vb_ followed by a number, a colon, and then each stat name.

cursor_name:cursor_checkpoint_idCheckpoint ID at which the cursor is
name ‘cursor_name’ is pointing now
cursor_name:cursor_seqnoThe seqno at which the cursor
‘cursor_name’ is pointing now
open_checkpoint_idID of the current open checkpoint
num_tap_cursorsNumber of referencing TAP cursors
num_checkpoint_itemsNumber of total items in a checkpoint
datastructure
num_open_checkpoint_itemsNumber of items in the open checkpoint
num_checkpointsNumber of checkpoints in a checkpoint
datastructure
num_items_for_persistenceNumber of items remaining for persistence
stateThe state of the vbucket this checkpoint
contains data for
last_closed_checkpoint_idThe last closed checkpoint number
persisted_checkpoint_idThe slast persisted checkpoint number

2.12 Memory Stats

This provides various memory-related stats including the stats from tcmalloc. Note that tcmalloc stats are not available on some operating systems (e.g., Windows) that do not support tcmalloc.

mem_used (deprecated)Engine’s total memory usage
bytesEngine’s total memory usage
ep_kv_sizeMemory used to store item metadata,
keys and values, no matter the
vbucket’s state. If an item’s value
is ejected, this stat will be
decremented by the size of the
item’s value.
ep_value_sizeMemory used to store values for
resident keys
ep_overheadExtra memory used by transient data
like persistence queue, replication
queues, checkpoints, etc
ep_max_sizeMax amount of data allowed in memory
ep_mem_low_watLow water mark for auto-evictions
ep_mem_high_watHigh water mark for auto-evictions
ep_oom_errorsNumber of times unrecoverable OOMs
happened while processing operations
ep_tmp_oom_errorsNumber of times temporary OOMs
happened while processing operations
ep_blob_numThe number of blob objects in the
cache
ep_blob_overheadThe “unused” memory caused by the
allocator returning bigger chunks
than requested
ep_storedval_sizeMemory used by storedval objects
ep_storedval_overheadThe “unused” memory caused by the
allocator returning bigger chunks
than requested
ep_storedval_numThe number of storedval objects
allocated
ep_item_numThe number of item objects allocated
ep_mem_tracker_enabledIf smart memory tracking is enabled
total_allocated_bytesEngine’s total memory usage reported
from the underlying memory allocator
total_heap_sizeBytes of system memory reserved by
the underlying memory allocator
total_free_mapped_bytesNumber of bytes in free, mapped
pages in the underlying allocator’s
page heap
total_free_unmapped_bytesNumber of bytes in free, unmapped
pages in page heap. These are bytes
that have been released back to OS
by the underlying memory allocator
total_fragmentation_bytesBytes of the fragmented memory in
the underlying allocator. Note that
the free and mapped pages inside the
allocator are not considered as the
fragmentation as they can be used
for incoming memory allocations.
tcmalloc_max_thread_cache_bytesA limit to how much memory the
underlying memory allocator TCMalloc
dedicates for small objects
tcmalloc_current_thread_cache_bytesA measure of some of the memory that
the underlying allocator TCMalloc is
using for small objects

2.13 Stats Key and Vkey

key_casThe keys current cas valueKV
key_data_ageHow long the key has waited for itsKV
value to be persisted (0 if clean)KV
key_exptimeExpiration time from the epochKV
key_flagsFlags for this keyKV
key_is_dirtyIf the value is not yet persistedKV
key_last_modified_timeLast updated timeKV
key_validSee description belowV
key_vb_stateThe vbucket state of this keyKV

key_valid can have the following responses:

this_is_a_bug - Some case we didn’t take care of. dirty - The value in memory has not been persisted yet. length_mismatch - The key length in memory doesn’t match the length on disk. data_mismatch - The data in memroy doesn’t match the data on disk. flags_mismatch - The flags in memory don’t match the flags on disk. valid - The key is both on disk and in memory ram_but_not_disk - The value doesn’t exist yet on disk. item_deleted - The item has been deleted.

2.14 Warmup

Stats warmup shows statistics related to warmup logic

ep_warmupShows if warmup is enabled / disabled
ep_warmup_estimated_key_countEstimated number of keys in database
ep_warmup_estimated_value_countEstimated number of values in database
ep_warmup_stateThe current state of the warmup thread
ep_warmup_threadWarmup thread status
ep_warmup_key_countNumber of keys warmed up
ep_warmup_value_countNumber of values warmed up
ep_warmup_dupsDuplicates encountered during warmup
ep_warmup_oomOOMs encountered during warmup
ep_warmup_item_expiredNumber of items expired during warmup
ep_warmup_timeTime (µs) spent by warming data
ep_warmup_keys_timeTime (µs) spent by warming keys
ep_warmup_mutation_logNumber of keys present in mutation log
ep_warmup_access_logNumber of keys present in access log
ep_warmup_min_items_thresholdPercentage of total items warmed up
before we enable traffic
ep_warmup_min_memory_thresholdPercentage of max mem warmed up before
we enable traffic

2.15 KV Store Stats

These provide various low-level stats and timings from the underlying KV storage system and useful to understand various states of the storage system.

The following stats are available for all database engine:

openNumber of database open operations
closeNumber of database close operations
readTimeTime spent in read operations
readSizeSize of data in read operations
writeTimeTime spent in write operations
writeSizeSize of data in write operations
deleteTime spent in delete() calls

The following stats are available for the CouchStore database engine:

backend_typeType of backend database engine
commitTime spent in CouchStore commit operation
compactionTime spent in compacting vbucket database file
numLoadedVbNumber of Vbuckets loaded into memory
lastCommDocsNumber of docs in the last commit
failure_setNumber of failed set operation
failure_getNumber of failed get operation
failure_vbsetNumber of failed vbucket set operation
save_documentsTime spent in CouchStore save documents operation

2.16 KV Store Timing Stats

KV Store Timing stats provide timing information from the underlying storage system. These stats are on shard (group of partitions) level.

2.16.1 Available Stats

The following histograms are available from “kvtimings” in the form described in Timings section above. These stats are prefixed with the rw_<Shard number>: indicating the times spent doing various things:

committime spent in commit operations
compacttime spent in file compaction operations
deletetime spent in delete operations
save_documentstime spent in persisting documents in storage
writeTimetime spent in writing to storage subsystem
writeSizesizes of writes given to storage subsystem
bulkSizebatch sizes of the save documents calls
fsReadTimetime spent in doing filesystem reads
fsWriteTimetime spent in doing filesystem writes
fsSyncTimetime spent in doing filesystem sync operations
fsReadSizesizes of various filesystem reads issued
fsWriteSizesizes of various filesystem writes issued
fsReadSeekvalues of various seek operations in file

2.17 Workload Raw Stats

Some information about the number of shards and Executor pool information. These are available as “workload” stats:

ep_workload:num_shardsnumber of shards or groups of partitions
ep_workload:num_writersnumber of threads that prioritize write ops
ep_workload:num_readersnumber of threads that prioritize read ops
ep_workload:num_auxionumber of threads that prioritize aux io ops
ep_workload:num_nonionumber of threads that prioritize non io ops
ep_workload:max_writersmax number of threads doing write ops
ep_workload:max_readersmax number of threads doing read ops
ep_workload:max_auxiomax number of threads doing aux io ops
ep_workload:max_noniomax number of threads doing non io ops
ep_workload:num_sleepersnumber of threads that are sleeping
ep_workload:ready_tasksnumber of global tasks that are ready to run

Additionally the following stats on the current state of the TaskQueues are also presented

HiPrioQ_Writer:InQsizecount high priority bucket writer tasks waiting
HiPrioQ_Writer:OutQsizecount high priority bucket writer tasks runnable
HiPrioQ_Reader:InQsizecount high priority bucket reader tasks waiting
HiPrioQ_Reader:OutQsizecount high priority bucket reader tasks runnable
HiPrioQ_AuxIO:InQsizecount high priority bucket auxio tasks waiting
HiPrioQ_AuxIO:OutQsizecount high priority bucket auxio tasks runnable
HiPrioQ_NonIO:InQsizecount high priority bucket nonio tasks waiting
HiPrioQ_NonIO:OutQsizecount high priority bucket nonio tasks runnable
LowPrioQ_Writer:InQsizecount low priority bucket writer tasks waiting
LowPrioQ_Writer:OutQsizecount low priority bucket writer tasks runnable
LowPrioQ_Reader:InQsizecount low priority bucket reader tasks waiting
LowPrioQ_Reader:OutQsizecount low priority bucket reader tasks runnable
LowPrioQ_AuxIO:InQsizecount low priority bucket auxio tasks waiting
LowPrioQ_AuxIO:OutQsizecount low priority bucket auxio tasks runnable
LowPrioQ_NonIO:InQsizecount low priority bucket nonio tasks waiting
LowPrioQ_NonIO:OutQsizecount low priority bucket nonio tasks runnable

2.18 Dispatcher Stats/JobLogs

This provides the stats from AUX dispatcher and non-IO dispatcher, and from all the reader and writer threads running for the specific bucket. Along with stats, the job logs for each of the dispatchers and worker threads is also made available.

The following stats are available for the workers and dispatchers:

stateThreads’s current status: running, sleeping etc.
runtimeThe amount of time since the thread started running
taskThe activity/job the thread is involved with at the moment

The following stats are for individual job logs:

starttimeThe timestamp when the job started
runtimeTime it took for the job to run
taskThe activity/job the thread ran during that time

2.19 Stats Reset

Resets the list of stats below.

Reset Stats:

ep_bg_load
ep_bg_wait
ep_bg_max_load
ep_bg_min_load
ep_bg_max_wait
ep_bg_min_wait
ep_commit_time
ep_flush_duration
ep_flush_duration_highwat
ep_io_num_read
ep_io_num_write
ep_io_read_bytes
ep_io_write_bytes
ep_items_rm_from_checkpoints
ep_num_eject_failures
ep_num_pager_runs
ep_num_not_my_vbuckets
ep_num_value_ejects
ep_pending_ops_max
ep_pending_ops_max_duration
ep_pending_ops_total
ep_storage_age
ep_storage_age_highwat
ep_too_old
ep_too_young
ep_tap_bg_load_avg
ep_tap_bg_max_load
ep_tap_bg_max_wait
ep_tap_bg_min_load
ep_tap_bg_min_wait
ep_tap_bg_wait_avg
ep_tap_throttled
ep_tap_total_fetched
ep_vbucket_del_max_walltime
pending_ops

Reset Histograms:

bg_load
bg_wait
bg_tap_load
bg_tap_wait
chk_persistence_cmd
data_age
del_vb_cmd
disk_insert
disk_update
disk_del
disk_vb_del
disk_commit
get_stats_cmd
item_alloc_sizes
get_vb_cmd
notify_io
pending_ops
set_vb_cmd
storage_age
tap_mutation
tap_vb_reset
tap_vb_set

3 Details

3.1 Ages

The difference between ep_storage_age and ep_data_age is somewhat subtle, but when you consider that a given record may be updated multiple times before hitting persistence, it starts to be clearer.

ep_data_age is how old the data we actually wrote is.

ep_storage_age is how long the object has been waiting to be persisted.

3.2 Warming Up

Opening the data store is broken into three distinct phases:

3.2.1 Initializing

During the initialization phase, the server is not accepting connections or otherwise functional. This is often quick, but in a server crash can take some time to perform recovery of the underlying storage.

This time is made available via the ep_dbinit stat.

3.2.2 Warming Up

After initialization, warmup begins. At this point, the server is capable of taking new writes and responding to reads. However, only records that have been pulled out of the storage or have been updated from other clients will be available for request.

(note that records read from persistence will not overwrite new records captured from the network)

During this phase, ep_warmup_thread will report running and ep_warmed_up will be increasing as records are being read.

3.2.3 Complete

Once complete, ep_warmed_up will stop increasing and ep_warmup_thread will report complete.

4 Uuid

The uuid stats allows clients to check if the unique identifier created and assigned to the bucket when it is created. By looking at this a client can verify that the bucket hasn’t been recreated since it was used.