Skip to main content
Skip to main content

Monitor Metrics

Monitor Metrics

Doris FE process and BE processes provide complete monitoring metrics. Monitoring metrics can be divided into two categories:

  1. Process monitoring: mainly displays some monitoring values of the Doris process itself .
  2. Node monitoring: mainly displays the monitoring of the node machine itself where the Doris process is located, such as CPU , memory, IO , network , etc.

You can obtain the current monitoring by accessing the http port of the FE or BE node . like :

curl http://fe_host:http_port/metrics
curl http://be_host:webserver_port/metrics

Monitoring metrics are generated in a Prometheus compatible format, for eg:

doris_fe_cache_added{type="partition"} 0
doris_fe_cache_added{type="sql"} 0
doris_fe_cache_hit{type="partition"} 0
doris_fe_cache_hit{type="sql"} 0
doris_fe_connection_total 2

Monitoring metrics in Json format can be fetched using type parameter in rest interface, for eg:

curl http://fe_host:http_port/metrics?type=json
curl http://be_host:webserver_port/metrics?type=json

Monitoring levels and best practices

The last column in the table marks the importance level of the monitoring items. P0 means the most important, the larger the value , the lower the importance.

The vast majority of monitoring metric types are Counter . That is the cumulative value. You can obtain effective information by collecting monitoring values at intervals (such as every 15 seconds) and calculating the slope per unit time .

The query error rate can be obtained by calculating the slope of doris_fe_query_err, presented as (error per second).

FE Monitoring Metrics

Process monitoring

nameLabelunitDescriptionImplicationGrade
doris_fe_cache_added{type="partition"}NumCumulative value of the number of new Partition Cache
{type=" sql "}NumCumulative value of the number of new SQL Cache
doris_fe_cache_hit{type="partition"}NumCount of partition cache hits
{type=" sql "}NumCount of SQL Cache hits
doris_fe_connection_totalNumCurrent number of FE MySQL port connectionsUsed to monitor the number of query connections. If the number of connections exceeds the limit, new connections will not be accessible .P0
doris_fe_counter_hit_sql_block_ruleNumNumber of queries blocked by SQL BLOCK RULE
doris_fe_edit_log_clean{type="failed"}Numto clear historical metadata logsIt should not fail. If it fails, manual intervention is required.P0
{type="success"}NumThe number of times the historical metadata log was successfully cleared
doris_fe_edit_log{type=" accumulated_bytes "}bytevalue of metadata log write amountThe write rate can be obtained by calculating the slope to observe whether there is a delay in metadata writing.P0
{type=" current_bytes "}byteMetadata log current valueUsed to monitor editlog size. If the size exceeds the limit, manual intervention is requiredP0
{type="read"}NumCount of metadata log readsObserve whether the metadata reading frequency is normal through the slopeP0
{type="write"}NumCount of metadata log writesObserve whether the metadata writing frequency is normal through the slopeP0
{type="current"}NumCurrent number of metadata logsUsed to monitor editlog quantity. If the quantity exceeds the limit, manual intervention is requiredP0
doris_fe_editlog_write_latency_msmillisecondmetadata log write latency . For example, {quantile="0.75"} indicates the 75th percentile write latency .
doris_fe_image_clean{type="failed"}Numto clean historical metadata image filesIt should not fail. If it fails, manual intervention is required.P0
{type="success"}NumThe number of times successful cleaning of historical metadata image files
doris_fe_image_push{type="failed"}NumThe number of failures in pushing metadata image files to other FE nodes
{type="success"}Nummetadata image files to other FE nodes
doris_fe_image_write{type="failed"}NumNumber of failures to generate metadata image filesIt should not fail. If it fails, manual intervention is required.P0
{type="success"}Numthe metadata image file was successfully generated
doris_fe_jobNumCurrent count of different job types and different job statuses. For example, {job="load", type="INSERT", state="LOADING"} represents an import job of type INSERT and the number of jobs in the LOADING state .observe the number of different types of jobs in the cluster as neededP0
doris_fe_max_journal_idNumThe maximum metadata log ID of the current FE node . If it is Master FE , it is the maximum ID currently written , if it is a non- Master FE , represents the maximum ID of the metadata log currently being played backUsed to observe whether the ID gap between multiple FEs is too large. If it is too large, it indicates a problem with metadata synchronization.P0
doris_fe_max_tablet_compaction_scoreNumThe largest compaction score value among all BE nodes .This value can be used to observe the maximum compaction score of the current cluster to determine whether it is too high. If it is too high, query or write delays may occur.P0
doris_fe_qpsNum/SecCurrent number of FE queries per second ( only query requests are counted )QPSP0
doris_fe_query_errNumvalue of error query
doris_fe_query_err_rateNum/SecError queries per secondObserve whether query errors occur in the clusterP0
doris_fe_query_latency_msmillisecondPercentile statistics of query request latency. For example, {quantile="0.75"} indicates the query delay at the 75th percentileDetailed observation of query latency in each quantileP0
doris_fe_query_latency_ms_dbmillisecondPercentile statistics of query request delay of each DB . For example, {quantile="0.75 ",db ="test"} indicates the query delay of the 75th percentile of DB testObserve the query latency of each DB in detailP0
doris_fe_query_olap_tableNumthe statistics of the number of requests for the internal table ( OlapTable )
doris_fe_query_totalNumall query requests
doris_fe_report_queue_sizeNumThe queue length of various periodic reporting tasks of BE on the FE sideThis value reflects the blocking degree of the reporting task on the Master FE node. The larger the value, the lower the processing capacity of the FE .P0
doris_fe_request_totalNumAll operation requests received through the MySQL port (including queries and other statements )
doris_fe_routine_load_error_rowsNumCount the total number of error rows for all Routine Load jobs in the cluster
doris_fe_routine_load_receive_bytesbytethe amount of data received by all Routine Load jobs in the cluster
doris_fe_routine_load_rowsNumCount the number of data rows received by all Routine Load jobs in the cluster
doris_fe_rpsNumcurrent number of FE requests per second (including queries and other types of statements )Work with QPS to view the amount of requests processed by the cluster .P0
doris_fe_scheduled_tablet_numNumtablets being scheduled by the Master FE node . Includes replicas being repaired and replicas being balancedtablets being migrated . If there is a value for a long time, it means the cluster is unstable.P0
doris_fe_tablet_max_compaction_scoreNumThe compaction core reported by each BE node . For example, { backend="172.21.0.1:9556"} represents the reported value of BE "172.21.0.1:9556"
doris_fe_tablet_numNumcurrent total number of tablets on each BE node . For example, {backend="172.21.0.1:9556"} indicates the current number of tablets of the BE "172.21.0.1:9556"You can check whether the tablet distribution is uniform and whether the absolute value is reasonableP0
doris_fe_tablet_status_countNumStatistics Master FE node The cumulative value of the number of tablets scheduled by the tablet scheduler .
{type="added"}NumStatistics Master FE node The cumulative value of the number of tablets scheduled by the tablet scheduler . "added" indicates the number of tablets that have been scheduled
{type=" in_sched "}NumSame as above. Indicates the number of tablets that are scheduled repeatedlyIf this value increases quickly, it means that a tablet has been in an unhealthy state for a long time , causing it to be scheduled repeatedly by the scheduler.
{type=" not_ready "}NumSame as above. Indicates the number of tablets that have not yet met the scheduling trigger conditions .If this value increases quickly, it means that a large number of tablets are in an unhealthy state but cannot be scheduled.
{type="total"}NumSame as above. Represents the cumulative number of tablets that have been checked (but not necessarily scheduled) .
{type="unhealthy"}NumSame as above. Indicates the cumulative number of unhealthy tablets that have been checked .
doris_fe_thread_poolNumCount the number of working threads and queuing status of various thread pools . "active_thread_num" Indicates the number of tasks being executed . "pool_size" Indicates the total number of threads in the thread pool . "task_in_queue" Indicates the number of tasks being queued
{name="agent-task-pool"}NumMaster FE is used to send Agent Task to BE 's thread pool
{name="connect-scheduler-check-timer"}Numpool for checking if MySQL idle connection has timed out
{name="connect-scheduler-pool"}Numpool for receiving MySQL connection requests
{name=" mysql - nio -pool"}NumNIO MySQL Server thread pool for processing tasks
{name="export-exporting-job-pool"}Numpool for export jobs in exporting state
{name="export-pending-job-pool"}Numpool for export jobs in pending state
{name="heartbeat- mgr -pool"}NumMaster FE is used to process the thread pool of each node's heartbeat
{name="loading-load-task-scheduler"}NumMaster FE is used to schedule Broker In Load job, loading Task scheduling thread pool
{name="pending-load-task-scheduler"}NumMaster FE is used to schedule Broker Load job , pending Task scheduling thread pool
{name="schema-change-pool"}Numpool used by Master FE to schedule schema change jobs
{name="thrift-server-pool"}Numworker thread pool of ThriftServer on the FE side . correspond fe.conf middle rpc_port . Used to interact with BE .
doris_fe_txn_counterNumvalue of the number of imported transactions in each statusYou can observe the execution of imported transactions .P0
{type="begin"}NumNumber of transactions committed
{type="failed"}NumNumber of failed transactions
{type="reject"}Numrejected transactions . ( If the number of currently running transactions is greater than the threshold, new transactions will be rejected )
{type=" succes "}Numnumber of successful transactions
doris_fe_txn_statusNumCount the number of import transactions currently in various states. For example, {type="committed"} indicates the number of transactions in the committed state.You can observe the number of imported transactions in each state to determine whether there is accumulation.P0
doris_fe_query_instance_numNumSpecifies the fragment that the user is currently requesting Number of instances . For example, {user=" test_u "} represents the user test_u The number of instances currently being requestedThis value can be used to observe whether the specified user takes up too many query resources .P0
doris_fe_query_instance_beginNumSpecify the fragment where the user request starts Number of instances . For example, {user=" test_u "} represents the user test_u Number of instances to start requestingThis value can be used to observe whether a given user has submitted too many queries.P0
doris_fe_query_rpc_totalNumof RPCs sent to the specified BE . For example, { be="192.168.10.1"} indicates the number of RPCs sent to BE with IP address 192.168.10.1This value can be used to observe whether too many RPCs are submitted to a certain BE .
doris_fe_query_rpc_failedNumRPC failures sent to the specified BE . For example, { be="192.168.10.1"} indicates the number of RPC failures sent to BE with IP address 192.168.10.1This value can be used to observe whether a certain BE has RPC problems.
doris_fe_query_rpc_sizeNumSpecify the RPC data size of BE . For example, { be="192.168.10.1"} indicates the number of RPC data bytes sent to BE with IP address 192.168.10.1This value can be used to observe whether an excessively large RPC is submitted to a BE .
doris_fe_txn_exec_latency_msmillisecondPercentile statistics of transaction execution time. For example, {quantile="0.75"} indicates the 75th percentile transaction execution timeObserve the execution time of each digit transaction in detailP0
doris_fe_txn_publish_latency_msmillisecondPercentile statistics of transaction publish time. For example, {quantile="0.75"} indicates that the 75th percentile transaction publish time isof the publishing time of each quantile transactionP0
doris_fe_txn_numNumSpecifies the number of transactions being performed by the DB . For example, { db ="test"} indicates the number of transactions currently being executed by DB test .This value can be used to observe whether a certain DB has submitted a large number of transactions .P0
doris_fe_publish_txn_numNumSpecify the number of transactions being published by the DB . For example, { db ="test"} indicates the number of transactions currently being published by DB test .This value can be used to observe the number of publish transactions of a certain DB .P0
doris_fe_txn_replica_numNumSpecifies the number of replicas opened by the transaction being executed by the DB . For example, { db ="test"} indicates the number of copies opened by the transaction currently being executed by DB test .This value can be used to observe whether a certain DB has too many copies opened, which may affect the execution of other transactions.P0
doris_fe_thrift_rpc_totalNumRPC requests received by each method of the FE thrift interface . For example, {method="report"} indicates the number of RPC requests received by the report method.This value can observe a certain thrift rpc method payload
doris_fe_thrift_rpc_latency_msmillisecondThe RPC requests received by each method of the FE thrift interface take time. For example, {method="report"} indicates that the RPC request received by the report method takes time.This value can observe a certain thrift rpc method payload
doris_fe_external_schema_cache{ catalog ="hive"}NumSpecifyExternal Catalog _ The number of corresponding schema caches
doris_fe_hive_meta_cache{ catalog ="hive"}Num
{type=" partition_value "}NumSpecify External Hive Metastore Catalog The number of corresponding partition value caches
{type="partition"}NumSpecify External Hive Metastore Catalog The number of corresponding partition caches
{type="file"}NumSpecify External Hive Metastore Catalog The number of corresponding file caches

JVM metrics

nameLabelunitDescriptionImpactGrade
jvm_heap_size_bytesbyteJVM memory metrics. The tags include max, used, committed , corresponding to the maximum value, used and requested memory respectively.Observe JVM memory usageP0
jvm_non_heap_size_bytesbyteJVM off-heap memory statistics
<GarbageCollector>GC metrics .GarbageCollector refers to a specific garbage collectorP0
{type="count"}NumCumulative value of GC times
{type="time"}millisecondCumulative value of GC time consumption
jvm_old_size_bytesbyteJVM old generation memory statisticsP0
jvm_threadNumJVM thread count statisticsObserve whether the number of JVM threads is reasonableP0
jvm_young_size_bytesbyteJVM new generation memory statisticsP0

Machine metrics

nameLabelunitDescriptionImpactGrade
system_meminfobyteFE node machines. Collected from /proc/meminfo . include buffers , cached , memory_available , memory_free , memory_total
system_snmpFE node machines. Collected from /proc/net/ snmp .
{name=" tcp_in_errs "}Numtcp packet reception errors
{name=" tcp_in_segs "}Numtcp packets sent
{name=" tcp_out_segs "}Numtcp packets sent
{name=" tcp_retrans_segs "}NumNumber of tcp packet retransmissions

BE metrics metrics

Process metrics

nameLabelunitDescriptionImpactgrade
doris_be_active_scan_context_countNumthe number of scanners currently opened directly from the outside
doris_be_add_batch_task_queue_sizeNumWhen recording import, the queue size of the thread pool that receives the batchIf it is greater than 0 , it means there is a backlog at the receiving end of the import task.P0
agent_task_queue_sizeNumDisplay the length of each Agent Task processing queue, such as {type="CREATE_TABLE"} Indicates the length of the CREATE_TABLE task queue
doris_be_brpc_endpoint_stub_countNumCreated _ The number of brpc stubs used for interaction between BEs
doris_be_brpc_function_endpoint_stub_countNumCreated _ The number of brpc stubs used to interact with Remote RPC
doris_be_cache_capacityRecord the capacity of the specified LRU Cache
doris_be_cache_usageRecord the usage of the specified LRU CacheUsed to observe memory usageP0
doris_be_cache_usage_ratioRecord the usage of the specified LRU Cache
doris_be_cache_lookup_countRecord the number of times the specified LRU Cache is searched
doris_be_cache_hit_countRecord the number of hits in the specified LRU Cache
doris_be_cache_hit_ratioRecord the hit rate of the specified LRU CacheUsed to observe whether the cache is effectiveP0
{name=" DataPageCache "}NumDataPageCache Data Page for caching dataData Cache , directly affects query efficiencyP0
{name=" IndexPageCache "}NumIndexPageCache Index Page for caching dataIndex Cache , directly affects query efficiencyP0
{name=" LastestSuccessChannelCache "}NumLastestSuccessChannelCache Used to cache import receivers LoadChannel
{name=" SegmentCache "}NumSegmentCache Used to cache turned on Segment , such as index information
doris_be_chunk_pool_local_core_alloc_countNumChunkAllocator , the number of times memory is allocated from the memory queue of the bound core
doris_be_chunk_pool_other_core_alloc_countNumChunkAllocator , the number of times memory is allocated from the memory queue of other cores
doris_be_chunk_pool_reserved_bytesbyteChunkAllocator The amount of memory reserved in
doris_be_chunk_pool_system_alloc_cost_nsnanosecondSystemAllocator The cumulative value of time spent applying for memoryThe slope can be used to observe the time taken for memory allocation.P0
doris_be_chunk_pool_system_alloc_countNumSystemAllocator Number of times to apply for memory
doris_be_chunk_pool_system_free_cost_nsnanosecondSystemAllocator Cumulative value of time taken to release memoryThe slope can be used to observe the time it takes to release memory.P0
doris_be_chunk_pool_system_free_countNumSystemAllocator The number of times memory is released
doris_be_compaction_bytes_totalbytevalue of the amount of data processed by compactionWhat is recorded is the input in the compaction task rowset The disk size . It can be observed through the slope rate of compactionP0
{type="base"}byteAccumulated data volume of Base Compaction
{type="cumulative"}byteThe cumulative data volume of Cumulative Compaction
doris_be_compaction_deltas_totalNumprocessed by compaction rowset The cumulative value of the numberWhat is recorded is the input in the compaction task rowset of number
{type="base"}NumProcessed by Base Compaction rowset Cumulative number
{type="cumulative"}NumProcessed by Cumulative Compaction rowset Cumulative number
doris_be_disks_compaction_numNumcompaction tasks being executed on the specified data directory . like {path="/path1/"} means /path1 The number of tasks being executed on the directoryUsed to observe whether the number of compaction tasks on each disk is reasonable.P0
doris_be_disks_compaction_scoreNumSpecifies the number of compaction tokens being executed on the data directory. like {path="/path1/"} means /path1 Number of tokens being executed on the directory
doris_be_compaction_used_permitsNumThe number of tokens used by the Compaction taskUsed to reflect the resource consumption of Compaction
doris_be_compaction_waitting_permitsNumCompaction tokens awaiting
doris_be_data_stream_receiver_countNumNumber of data receiving terminals ReceiverFIXME : This indicator is missing for the vectorization engine
doris_be_disks_avail_capacitybyteSpecify the remaining space on the disk where the specified data directory is located. like {path="/path1/"} express /path1 The remaining space on the disk where the directory is locatedP0
doris_be_disks_local_used_capacitybytethe specified data directory is located
doris_be_disks_remote_used_capacitybytethe specified data directory is located
doris_be_disks_stateBooleanSpecifies the disk status of the data directory . 1 means normal. 0 means abnormal
doris_be_disks_total_capacitybytecapacity of the disk where the specified data directory is locatedCooperate doris_be_disks_avail_capacity Calculate disk usageP0
doris_be_engine_requests_totalNumvalue of various task execution statuses on BE
{status=" failed ",type ="xxx"}Numvalue of the number of failures for tasks of type xxx
{status=" total ",type ="xxx"}NumThe cumulative value of the total number of tasks of type xxx .Can monitor the number of failures of various tasks on demandP0
{status=" skip ",type =" report_all_tablets "}Numvalue of the number of times xxx type tasks have been skipped
doris_be_fragment_endpoint_countNumsameFIXME: Same as doris_be_data_stream_receiver_count number. And the vectorization engine is missing
doris_be_fragment_request_duration_usmicrosecondsAll fragment intance The cumulative execution time ofthe execution time of instance through slopeP0
doris_be_fragment_requests_totalNumThe cumulative number of executed fragment instances
doris_be_load_channel_countNumThe number of load channels currently openThe larger the value , the more import tasks are currently being executed.P0
doris_be_local_bytes_read_totalbyteDepend on LocalFileReader Number of bytes readP0
doris_be_local_bytes_written_totalbyteDepend on LocalFileWriter Number of bytes writtenP0
doris_be_local_file_reader_totalNumopened LocalFileReader Cumulative count of
doris_be_local_file_open_readingNumcurrently open LocalFileReader number
doris_be_local_file_writer_totalNumopened LocalFileWriter cumulative count .
doris_be_mem_consumptionbyteSpecifies the current memory overhead of the module . For example, {type="compaction"} represents the current total memory overhead of the compaction module .Values taken from the same type MemTracker . FIXME
doris_be_memory_allocated_bytesbyteBE process physical memory size, taken from /proc/self/status/ VmRSSP0
doris_be_memory_jemallocbyteJemalloc stats, taken from je_mallctl .Meaningreference : https://jemalloc.net/jemalloc.3.htmlP0
doris_be_memory_pool_bytes_totalbyteall MemPool The size of memory currently occupied. Statistical value, does not represent actual memory usage .
doris_be_memtable_flush_duration_usmicrosecondsvalue of the time taken to write memtable to diskWrite latency can be observed via slopeP0
doris_be_memtable_flush_totalNumnumber of memtable writes to diskThe slope can be used to calculate the frequency of writing to a fileP0
doris_be_meta_request_durationmicrosecondsaccess RocksDB The cumulative time consumption of meta inBE metadata read and write latency through slopeP0
{type="read"}microsecondsReading time
{type="write"}microsecondsWriting time
doris_be_meta_request_totalNumaccess RocksDB The cumulative number of meta inBE metadata access frequency by slopeP0
{type="read"}NumRead times
{type="write"}NumNumber of writes
doris_be_fragment_instance_countNumThe number of fragment instances currently receivedObserve whether instance accumulation occursP0
doris_be_process_fd_num_limit_hardNumBE process. pass /proc/ pid /limits collection
doris_be_process_fd_num_limit_softNumBE process. pass /proc/ pid /limits collection
doris_be_process_fd_num_usedNumThe number of file handles used by the BE process. pass /proc/ pid /limits collection
doris_be_process_thread_numNumBE process threads. pass /proc/ pid /task collectionP0
doris_be_query_cache_memory_total_bytebyteNumber of bytes occupied by Query Cache
doris_be_query_cache_partition_total_countNumcurrent number of Partition Cache caches
doris_be_query_cache_sql_total_countNumCurrent number of SQL Cache caches
doris_be_query_scan_bytesbyteRead the cumulative value of the data amount. Here we only count reads Olap The amount of data in the table
doris_be_query_scan_bytes_per_secondBytes / secondaccording to doris_be_query_scan_bytes Calculated read rateObserve query rateP0
doris_be_query_scan_rowsNumRead the cumulative value of the number of rows. Here we only count reads Olap The amount of data in the table. and is RawRowsRead (Some data rows may be skipped by the index and not actually read, but will still be recorded in this value )Query rate by slope observationP0
doris_be_result_block_queue_countNumThe number of fragment instances in the current query result cacheThis queue is only used when being read directly by an external system. For example, Spark on Doris queries data through external scan.
doris_be_result_buffer_block_countNumThe number of queries in the current query result cacheThis value reflects how many query results are currently waiting for FE consumption in BE .P0
doris_be_routine_load_task_countNumThe number of routine load tasks currently being executed
doris_be_rowset_count_generated_and_in_useNumNew and in use since the last startup The number of rowset ids .
doris_be_s3_bytes_read_totalNumS3FileReader The cumulative number of opens
doris_be_s3_file_open_readingNumcurrently open S3FileReader number
doris_be_s3_bytes_read_totalbyteS3FileReader Read the cumulative value of bytes
doris_be_scanner_thread_pool_queue_sizeNumused for OlapScanner The current queued number of thread poolsit is greater than zero , it means that Scanner starts to accumulate.P0
doris_be_segment_read{type=" segment_read_total "}NumCumulative value of the number of segments read
doris_be_segment_read{type=" segment_row_total "}Numvalue of the number of rows in the segment readThis value also includes the number of rows filtered by the index. Equivalent to the number of segments read * the total number of rows in each segment
doris_be_send_batch_thread_pool_queue_sizeNumThe number of queues in the thread pool used to send data packets when importingit is greater than 0 , it means there is accumulationP0
doris_be_send_batch_thread_pool_thread_numNumThe number of threads in the thread pool used to send packets when importing
doris_be_small_file_cache_countNumcurrently cached by BE
doris_be_streaming_load_current_processingNumNumber of stream load tasks currently runningContains only tasks sent by the curl command
doris_be_streaming_load_duration_msmillisecondThe cumulative value of the execution time of all stream load tasks
doris_be_streaming_load_requests_totalNumvalue of the number of stream load tasksObservable task submission frequency via slopeP0
doris_be_stream_load_pipe_countNumcurrent stream load data pipelinesIncluding stream load and routine load tasks
doris_be_stream_load{type=" load_rows "}Numnumber of rows finally imported by stream loadIncluding stream load and routine load tasksP0
doris_be_stream_load{type=" receive_bytes "}bytevalue of the number of bytes received by stream loadIncluding stream load data received from http , and routine load from kafka read dataP0
doris_be_tablet_base_max_compaction_scoreNumThe current largest Base Compaction ScoreThis value changes in real time, and peak data may be lost. The higher the value, the more serious the compaction accumulation is .P0
doris_be_tablet_cumulative_max_compaction_scoreNumSame as above. Current largest Cumulative Compaction Score
doris_be_tablet_version_num_distributionNumThe histogram of the number of tablet versions .Distribution used to reflect the number of tablet versionsP0
doris_be_thrift_connections_totalNumthrift connections created . like {name="heartbeat"} Indicates the cumulative number of connections to the heartbeat serviceThis value is the connection of thrift server with BE as the server.
doris_be_thrift_current_connectionsNumcurrent number of thrift connections. like {name="heartbeat"} Indicates the current number of connections to the heartbeat service .Same as above
doris_be_thrift_opened_clientsNumthrift clients currently open . like {name="frontend"} Indicates the number of clients accessing the FE service
doris_be_thrift_used_clientsNumthrift clients currently in use . like {name="frontend"} Indicates the number of clients being used to access the FE service
doris_be_timeout_canceled_fragment_countNumCumulative value of the number of fragment instances canceled due to timeoutThis value may be recorded repeatedly. For example, some fragment instances are canceled multiple timesP0
doris_be_stream_load_txn_request{type="begin"}Numvalue of the number of transactions started by stream loadIncluding stream load and routine load tasks
doris_be_stream_load_txn_request{type="commit"}Numvalue of the number of transactions successfully executed by stream loadSame as above
doris_be_stream_load_txn_request{type="rollback"}value of the number of transactions that failed to execute stream loadSame as above
doris_be_unused_rowsets_countNumThe number of currently abandoned rowsetsThese rowsets will be deleted regularly under normal circumstances.
doris_be_upload_fail_countNumcumulative value of rowset failed to be uploaded to remote storage
doris_be_upload_rowset_countNumcumulative number of rowsets successfully uploaded to remote storage
doris_be_upload_total_bytebytevalue of rowset data successfully uploaded to remote storage
doris_be_load_bytesbyteCumulative quantity sent through tablet sinkObservable amount of imported dataP0
doris_be_load_rowsNumCumulative number of rows sent through tablet sinkObservable amount of imported dataP0
fragment_thread_pool_queue_sizeNumcurrent query execution thread pool waiting queueIf it is greater than zero, the query thread has been exhausted and queries will pile up.P0
doris_be_all_rowsets_numNumAll currently rowset number ofP0
doris_be_all_segments_numNumThe number of all current segmentsP0
doris_be_heavy_work_max_threadsNumbrpc Number of heavy thread pool threadsp0
doris_be_light_work_max_threadsNumbrpc Number of light thread pool threadsp0
doris_be_heavy_work_pool_queue_sizeNumbrpc The maximum length of the heavy thread pool queue will block the submission of work if it exceeds it .p0
doris_be_light_work_pool_queue_sizeNumbrpc The maximum length of the light thread pool queue . If it exceeds the maximum length, the submission of work will be blocked.p0
doris_be_heavy_work_active_threadsNumbrpc Number of active threads in heavy thread poolp0
doris_be_light_work_active_threadsNumbrpc Number of active threads in light thread poolp0

Machine metrics

nameLabelunitDescriptionImpactGrade
doris_be_cpuNumCPU related metrics metrics, from /proc/stat collection. Each value of each logical core will be collected separately . like {device="cpu0 ",mode ="nice"} Indicates the nice value of cpu0CPU usage can be calculatedP0
doris_be_disk_bytes_readbyteThe cumulative value of disk reads. from /proc/ diskstats collection. The values of each disk will be collected separately . like {device=" vdd "} express vvd disk value
doris_be_disk_bytes_writtenbyteThe cumulative value of disk writes. The collection method is the same as above
doris_be_disk_io_time_msbyteThe collection method is the same as aboveIO Util can be calculatedP0
doris_be_disk_io_time_weightedbyteThe collection method is the same as above
doris_be_disk_reads_completedbyteThe collection method is the same as above
doris_be_disk_read_time_msbyteThe collection method is the same as above
doris_be_disk_writes_completedbyteThe collection method is the same as above
doris_be_disk_write_time_msbyteThe collection method is the same as above
doris_be_fd_num_limitNumSystem file handle limit upper limit. from /proc/sys/fs/file-nr collection
doris_be_fd_num_usedNumThe number of file handles used by the system . from /proc/sys/fs/file-nr collection
doris_be_file_created_totalNumCumulative number of local file creation timesall calls local_file_writer And finally close the file count
doris_be_load_averageNumMachine Load Avg Metric metrics. For example, {mode="15_minutes"} is 15 minutes Load AvgObserve the overall machine loadP0
doris_be_max_disk_io_util_percentpercentagevalue of the disk with the largest IO UTIL among all disksP0
doris_be_max_network_receive_bytes_rateBytes / secondThe maximum receive rate calculated among all network cardsP0
doris_be_max_network_send_bytes_rateBytes / secondThe calculated maximum sending rate among all network cardsP0
doris_be_memory_pgpginbyteThe amount of data written by the system from disk to memory page
doris_be_memory_pgpgoutbyteThe amount of data written to disk by system memory pages
doris_be_memory_pswpinbyteThe number of times the system swapped from disk to memoryNormally, swap should be turned off, so this value should be 0
doris_be_memory_pswpoutbyteThe number of times the system swapped from memory to diskNormally, swap should be turned off, so this value should be 0
doris_be_network_receive_bytesbyteeach network card are accumulated. Collected from /proc/net/dev
doris_be_network_receive_packetsNumeach network card is accumulated. Collected from /proc/net/dev
doris_be_network_send_bytesbyteeach network card . Collected from /proc/net/dev
doris_be_network_send_packetsNumThe total number of packets sent by each network card is accumulated. Collected from /proc/net/dev
doris_be_proc{mode=" ctxt_switch "}NumCPU context switches . Collected from /proc/statObserve whether there are abnormal context switchesP0
doris_be_proc{mode="interrupt"}NumCPU interrupts . Collected from /proc/stat
doris_be_proc{mode=" procs_blocked "}NumThe number of processes currently blocked in the system (such as waiting for IO ). Collected from /proc/stat
doris_be_proc{mode=" procs_running "}NumThe number of processes currently executing on the system . Collected from /proc/stat
doris_be_snmp_tcp_in_errsNumtcp packet reception errors. Collected from /proc/net/ snmpObservable network errors such as retransmission, packet loss, etc. Need and other snmp metrics used together withP0
doris_be_snmp_tcp_in_segsNumtcp packets sent . Collected from /proc/net/ snmp
doris_be_snmp_tcp_out_segsNumtcp packets sent. Collected from /proc/net/ snmp
doris_be_snmp_tcp_retrans_segsNumTCP packet retransmissions . Collected from /proc/net/ snmp