Monitor Metrics
Monitor Metrics
Doris FE process and BE processes provide complete monitoring metrics. Monitoring metrics can be divided into two categories:
- Process monitoring: mainly displays some monitoring values of the Doris process itself .
- 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
name | Label | unit | Description | Implication | Grade |
---|---|---|---|---|---|
doris_fe_cache_added | {type="partition"} | Num | Cumulative value of the number of new Partition Cache | ||
{type=" sql "} | Num | Cumulative value of the number of new SQL Cache | |||
doris_fe_cache_hit | {type="partition"} | Num | Count of partition cache hits | ||
{type=" sql "} | Num | Count of SQL Cache hits | |||
doris_fe_connection_total | Num | Current number of FE MySQL port connections | Used 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_rule | Num | Number of queries blocked by SQL BLOCK RULE | |||
doris_fe_edit_log_clean | {type="failed"} | Num | to clear historical metadata logs | It should not fail. If it fails, manual intervention is required. | P0 |
{type="success"} | Num | The number of times the historical metadata log was successfully cleared | |||
doris_fe_edit_log | {type=" accumulated_bytes "} | byte | value of metadata log write amount | The write rate can be obtained by calculating the slope to observe whether there is a delay in metadata writing. | P0 |
{type=" current_bytes "} | byte | Metadata log current value | Used to monitor editlog size. If the size exceeds the limit, manual intervention is required | P0 | |
{type="read"} | Num | Count of metadata log reads | Observe whether the metadata reading frequency is normal through the slope | P0 | |
{type="write"} | Num | Count of metadata log writes | Observe whether the metadata writing frequency is normal through the slope | P0 | |
{type="current"} | Num | Current number of metadata logs | Used to monitor editlog quantity. If the quantity exceeds the limit, manual intervention is required | P0 | |
doris_fe_editlog_write_latency_ms | millisecond | metadata log write latency . For example, {quantile="0.75"} indicates the 75th percentile write latency . | |||
doris_fe_image_clean | {type="failed"} | Num | to clean historical metadata image files | It should not fail. If it fails, manual intervention is required. | P0 |
{type="success"} | Num | The number of times successful cleaning of historical metadata image files | |||
doris_fe_image_push | {type="failed"} | Num | The number of failures in pushing metadata image files to other FE nodes | ||
{type="success"} | Num | metadata image files to other FE nodes | |||
doris_fe_image_write | {type="failed"} | Num | Number of failures to generate metadata image files | It should not fail. If it fails, manual intervention is required. | P0 |
{type="success"} | Num | the metadata image file was successfully generated | |||
doris_fe_job | Num | Current 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 needed | P0 | |
doris_fe_max_journal_id | Num | The 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 back | Used 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_score | Num | The 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_qps | Num/Sec | Current number of FE queries per second ( only query requests are counted ) | QPS | P0 | |
doris_fe_query_err | Num | value of error query | |||
doris_fe_query_err_rate | Num/Sec | Error queries per second | Observe whether query errors occur in the cluster | P0 | |
doris_fe_query_latency_ms | millisecond | Percentile statistics of query request latency. For example, {quantile="0.75"} indicates the query delay at the 75th percentile | Detailed observation of query latency in each quantile | P0 | |
doris_fe_query_latency_ms_db | millisecond | Percentile 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 test | Observe the query latency of each DB in detail | P0 | |
doris_fe_query_olap_table | Num | the statistics of the number of requests for the internal table ( OlapTable ) | |||
doris_fe_query_total | Num | all query requests | |||
doris_fe_report_queue_size | Num | The queue length of various periodic reporting tasks of BE on the FE side | This 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_total | Num | All operation requests received through the MySQL port (including queries and other statements ) | |||
doris_fe_routine_load_error_rows | Num | Count the total number of error rows for all Routine Load jobs in the cluster | |||
doris_fe_routine_load_receive_bytes | byte | the amount of data received by all Routine Load jobs in the cluster | |||
doris_fe_routine_load_rows | Num | Count the number of data rows received by all Routine Load jobs in the cluster | |||
doris_fe_rps | Num | current 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_num | Num | tablets being scheduled by the Master FE node . Includes replicas being repaired and replicas being balanced | tablets being migrated . If there is a value for a long time, it means the cluster is unstable. | P0 | |
doris_fe_tablet_max_compaction_score | Num | The 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_num | Num | current 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 reasonable | P0 | |
doris_fe_tablet_status_count | Num | Statistics Master FE node The cumulative value of the number of tablets scheduled by the tablet scheduler . | |||
{type="added"} | Num | Statistics 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 "} | Num | Same as above. Indicates the number of tablets that are scheduled repeatedly | If 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 "} | Num | Same 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"} | Num | Same as above. Represents the cumulative number of tablets that have been checked (but not necessarily scheduled) . | |||
{type="unhealthy"} | Num | Same as above. Indicates the cumulative number of unhealthy tablets that have been checked . | |||
doris_fe_thread_pool | Num | Count 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"} | Num | Master FE is used to send Agent Task to BE 's thread pool | |||
{name="connect-scheduler-check-timer"} | Num | pool for checking if MySQL idle connection has timed out | |||
{name="connect-scheduler-pool"} | Num | pool for receiving MySQL connection requests | |||
{name=" mysql - nio -pool"} | Num | NIO MySQL Server thread pool for processing tasks | |||
{name="export-exporting-job-pool"} | Num | pool for export jobs in exporting state | |||
{name="export-pending-job-pool"} | Num | pool for export jobs in pending state | |||
{name="heartbeat- mgr -pool"} | Num | Master FE is used to process the thread pool of each node's heartbeat | |||
{name="loading-load-task-scheduler"} | Num | Master FE is used to schedule Broker In Load job, loading Task scheduling thread pool | |||
{name="pending-load-task-scheduler"} | Num | Master FE is used to schedule Broker Load job , pending Task scheduling thread pool | |||
{name="schema-change-pool"} | Num | pool used by Master FE to schedule schema change jobs | |||
{name="thrift-server-pool"} | Num | worker thread pool of ThriftServer on the FE side . correspond fe.conf middle rpc_port . Used to interact with BE . | |||
doris_fe_txn_counter | Num | value of the number of imported transactions in each status | You can observe the execution of imported transactions . | P0 | |
{type="begin"} | Num | Number of transactions committed | |||
{type="failed"} | Num | Number of failed transactions | |||
{type="reject"} | Num | rejected transactions . ( If the number of currently running transactions is greater than the threshold, new transactions will be rejected ) | |||
{type=" succes "} | Num | number of successful transactions | |||
doris_fe_txn_status | Num | Count 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_num | Num | Specifies 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 requested | This value can be used to observe whether the specified user takes up too many query resources . | P0 | |
doris_fe_query_instance_begin | Num | Specify 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 requesting | This value can be used to observe whether a given user has submitted too many queries. | P0 | |
doris_fe_query_rpc_total | Num | of 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.1 | This value can be used to observe whether too many RPCs are submitted to a certain BE . | ||
doris_fe_query_rpc_failed | Num | RPC 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.1 | This value can be used to observe whether a certain BE has RPC problems. | ||
doris_fe_query_rpc_size | Num | Specify 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.1 | This value can be used to observe whether an excessively large RPC is submitted to a BE . | ||
doris_fe_txn_exec_latency_ms | millisecond | Percentile statistics of transaction execution time. For example, {quantile="0.75"} indicates the 75th percentile transaction execution time | Observe the execution time of each digit transaction in detail | P0 | |
doris_fe_txn_publish_latency_ms | millisecond | Percentile statistics of transaction publish time. For example, {quantile="0.75"} indicates that the 75th percentile transaction publish time is | of the publishing time of each quantile transaction | P0 | |
doris_fe_txn_num | Num | Specifies 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_num | Num | Specify 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_num | Num | Specifies 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_total | Num | RPC 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_ms | millisecond | The 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"} | Num | SpecifyExternal Catalog _ The number of corresponding schema caches | ||
doris_fe_hive_meta_cache | { catalog ="hive"} | Num | |||
{type=" partition_value "} | Num | Specify External Hive Metastore Catalog The number of corresponding partition value caches | |||
{type="partition"} | Num | Specify External Hive Metastore Catalog The number of corresponding partition caches | |||
{type="file"} | Num | Specify External Hive Metastore Catalog The number of corresponding file caches |
JVM metrics
name | Label | unit | Description | Impact | Grade |
---|---|---|---|---|---|
jvm_heap_size_bytes | byte | JVM memory metrics. The tags include max, used, committed , corresponding to the maximum value, used and requested memory respectively. | Observe JVM memory usage | P0 | |
jvm_non_heap_size_bytes | byte | JVM off-heap memory statistics | |||
jvm_old_gc | GC metrics . | GarbageCollector refers to a specific garbage collector | P0 | ||
{type="count"} | Num | Cumulative value of GC times | |||
{type="time"} | millisecond | Cumulative value of GC time consumption | |||
jvm_old_size_bytes | byte | JVM old generation memory statistics | P0 | ||
jvm_thread | Num | JVM thread count statistics | Observe whether the number of JVM threads is reasonable | P0 | |
jvm_young_size_bytes | byte | JVM new generation memory statistics | P0 |
Machine metrics
name | Label | unit | Description | Impact | Grade |
---|---|---|---|---|---|
system_meminfo | byte | FE node machines. Collected from /proc/meminfo . include buffers , cached , memory_available , memory_free , memory_total | |||
system_snmp | FE node machines. Collected from /proc/net/ snmp . | ||||
{name=" tcp_in_errs "} | Num | tcp packet reception errors | |||
{name=" tcp_in_segs "} | Num | tcp packets sent | |||
{name=" tcp_out_segs "} | Num | tcp packets sent | |||
{name=" tcp_retrans_segs "} | Num | Number of tcp packet retransmissions |
BE metrics metrics
Process metrics
name | Label | unit | Description | Impact | grade |
---|---|---|---|---|---|
doris_be_active_scan_context_count | Num | the number of scanners currently opened directly from the outside | |||
doris_be_add_batch_task_queue_size | Num | When recording import, the queue size of the thread pool that receives the batch | If it is greater than 0 , it means there is a backlog at the receiving end of the import task. | P0 | |
agent_task_queue_size | Num | Display 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_count | Num | Created _ The number of brpc stubs used for interaction between BEs | |||
doris_be_brpc_function_endpoint_stub_count | Num | Created _ The number of brpc stubs used to interact with Remote RPC | |||
doris_be_cache_capacity | Record the capacity of the specified LRU Cache | ||||
doris_be_cache_usage | Record the usage of the specified LRU Cache | Used to observe memory usage | P0 | ||
doris_be_cache_usage_ratio | Record the usage of the specified LRU Cache | ||||
doris_be_cache_lookup_count | Record the number of times the specified LRU Cache is searched | ||||
doris_be_cache_hit_count | Record the number of hits in the specified LRU Cache | ||||
doris_be_cache_hit_ratio | Record the hit rate of the specified LRU Cache | Used to observe whether the cache is effective | P0 | ||
{name=" DataPageCache "} | Num | DataPageCache Data Page for caching data | Data Cache , directly affects query efficiency | P0 | |
{name=" IndexPageCache "} | Num | IndexPageCache Index Page for caching data | Index Cache , directly affects query efficiency | P0 | |
{name=" LastestSuccessChannelCache "} | Num | LastestSuccessChannelCache Used to cache import receivers LoadChannel | |||
{name=" SegmentCache "} | Num | SegmentCache Used to cache turned on Segment , such as index information | |||
doris_be_chunk_pool_local_core_alloc_count | Num | ChunkAllocator , the number of times memory is allocated from the memory queue of the bound core | |||
doris_be_chunk_pool_other_core_alloc_count | Num | ChunkAllocator , the number of times memory is allocated from the memory queue of other cores | |||
doris_be_chunk_pool_reserved_bytes | byte | ChunkAllocator The amount of memory reserved in | |||
doris_be_chunk_pool_system_alloc_cost_ns | nanosecond | SystemAllocator The cumulative value of time spent applying for memory | The slope can be used to observe the time taken for memory allocation. | P0 | |
doris_be_chunk_pool_system_alloc_count | Num | SystemAllocator Number of times to apply for memory | |||
doris_be_chunk_pool_system_free_cost_ns | nanosecond | SystemAllocator Cumulative value of time taken to release memory | The slope can be used to observe the time it takes to release memory. | P0 | |
doris_be_chunk_pool_system_free_count | Num | SystemAllocator The number of times memory is released | |||
doris_be_compaction_bytes_total | byte | value of the amount of data processed by compaction | What is recorded is the input in the compaction task rowset The disk size . It can be observed through the slope rate of compaction | P0 | |
{type="base"} | byte | Accumulated data volume of Base Compaction | |||
{type="cumulative"} | byte | The cumulative data volume of Cumulative Compaction | |||
doris_be_compaction_deltas_total | Num | processed by compaction rowset The cumulative value of the number | What is recorded is the input in the compaction task rowset of number | ||
{type="base"} | Num | Processed by Base Compaction rowset Cumulative number | |||
{type="cumulative"} | Num | Processed by Cumulative Compaction rowset Cumulative number | |||
doris_be_disks_compaction_num | Num | compaction tasks being executed on the specified data directory . like {path="/path1/"} means /path1 The number of tasks being executed on the directory | Used to observe whether the number of compaction tasks on each disk is reasonable. | P0 | |
doris_be_disks_compaction_score | Num | Specifies 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_permits | Num | The number of tokens used by the Compaction task | Used to reflect the resource consumption of Compaction | ||
doris_be_compaction_waitting_permits | Num | Compaction tokens awaiting | |||
doris_be_data_stream_receiver_count | Num | Number of data receiving terminals Receiver | FIXME : This indicator is missing for the vectorization engine | ||
doris_be_disks_avail_capacity | byte | Specify 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 located | P0 | ||
doris_be_disks_local_used_capacity | byte | the specified data directory is located | |||
doris_be_disks_remote_used_capacity | byte | the specified data directory is located | |||
doris_be_disks_state | Boolean | Specifies the disk status of the data directory . 1 means normal. 0 means abnormal | |||
doris_be_disks_total_capacity | byte | capacity of the disk where the specified data directory is located | Cooperate doris_be_disks_avail_capacity Calculate disk usage | P0 | |
doris_be_engine_requests_total | Num | value of various task execution statuses on BE | |||
{status=" failed ",type ="xxx"} | Num | value of the number of failures for tasks of type xxx | |||
{status=" total ",type ="xxx"} | Num | The cumulative value of the total number of tasks of type xxx . | Can monitor the number of failures of various tasks on demand | P0 | |
{status=" skip ",type =" report_all_tablets "} | Num | value of the number of times xxx type tasks have been skipped | |||
doris_be_fragment_endpoint_count | Num | same | FIXME: Same as doris_be_data_stream_receiver_count number. And the vectorization engine is missing | ||
doris_be_fragment_request_duration_us | microseconds | All fragment intance The cumulative execution time of | the execution time of instance through slope | P0 | |
doris_be_fragment_requests_total | Num | The cumulative number of executed fragment instances | |||
doris_be_load_channel_count | Num | The number of load channels currently open | The larger the value , the more import tasks are currently being executed. | P0 | |
doris_be_local_bytes_read_total | byte | Depend on LocalFileReader Number of bytes read | P0 | ||
doris_be_local_bytes_written_total | byte | Depend on LocalFileWriter Number of bytes written | P0 | ||
doris_be_local_file_reader_total | Num | opened LocalFileReader Cumulative count of | |||
doris_be_local_file_open_reading | Num | currently open LocalFileReader number | |||
doris_be_local_file_writer_total | Num | opened LocalFileWriter cumulative count . | |||
doris_be_mem_consumption | byte | Specifies 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_bytes | byte | BE process physical memory size, taken from /proc/self/status/ VmRSS | P0 | ||
doris_be_memory_jemalloc | byte | Jemalloc stats, taken from je_mallctl . | Meaning reference : https://jemalloc.net/jemalloc.3.html | P0 | |
doris_be_memory_pool_bytes_total | byte | all MemPool The size of memory currently occupied. Statistical value, does not represent actual memory usage . | |||
doris_be_memtable_flush_duration_us | microseconds | value of the time taken to write memtable to disk | Write latency can be observed via slope | P0 | |
doris_be_memtable_flush_total | Num | number of memtable writes to disk | The slope can be used to calculate the frequency of writing to a file | P0 | |
doris_be_meta_request_duration | microseconds | access RocksDB The cumulative time consumption of meta in | BE metadata read and write latency through slope | P0 | |
{type="read"} | microseconds | Reading time | |||
{type="write"} | microseconds | Writing time | |||
doris_be_meta_request_total | Num | access RocksDB The cumulative number of meta in | BE metadata access frequency by slope | P0 | |
{type="read"} | Num | Read times | |||
{type="write"} | Num | Number of writes | |||
doris_be_fragment_instance_count | Num | The number of fragment instances currently received | Observe whether instance accumulation occurs | P0 | |
doris_be_process_fd_num_limit_hard | Num | BE process. pass /proc/ pid /limits collection | |||
doris_be_process_fd_num_limit_soft | Num | BE process. pass /proc/ pid /limits collection | |||
doris_be_process_fd_num_used | Num | The number of file handles used by the BE process. pass /proc/ pid /limits collection | |||
doris_be_process_thread_num | Num | BE process threads. pass /proc/ pid /task collection | P0 | ||
doris_be_query_cache_memory_total_byte | byte | Number of bytes occupied by Query Cache | |||
doris_be_query_cache_partition_total_count | Num | current number of Partition Cache caches | |||
doris_be_query_cache_sql_total_count | Num | Current number of SQL Cache caches | |||
doris_be_query_scan_bytes | byte | Read 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_second | Bytes / second | according to doris_be_query_scan_bytes Calculated read rate | Observe query rate | P0 | |
doris_be_query_scan_rows | Num | Read 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 observation | P0 | |
doris_be_result_block_queue_count | Num | The number of fragment instances in the current query result cache | This 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_count | Num | The number of queries in the current query result cache | This value reflects how many query results are currently waiting for FE consumption in BE . | P0 | |
doris_be_routine_load_task_count | Num | The number of routine load tasks currently being executed | |||
doris_be_rowset_count_generated_and_in_use | Num | New and in use since the last startup The number of rowset ids . | |||
doris_be_s3_bytes_read_total | Num | S3FileReader The cumulative number of opens | |||
doris_be_s3_file_open_reading | Num | currently open S3FileReader number | |||
doris_be_s3_bytes_read_total | byte | S3FileReader Read the cumulative value of bytes | |||
doris_be_scanner_thread_pool_queue_size | Num | used for OlapScanner The current queued number of thread pools | it is greater than zero , it means that Scanner starts to accumulate. | P0 | |
doris_be_segment_read | {type=" segment_read_total "} | Num | Cumulative value of the number of segments read | ||
doris_be_segment_read | {type=" segment_row_total "} | Num | value of the number of rows in the segment read | This 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_size | Num | The number of queues in the thread pool used to send data packets when importing | it is greater than 0 , it means there is accumulation | P0 | |
doris_be_send_batch_thread_pool_thread_num | Num | The number of threads in the thread pool used to send packets when importing | |||
doris_be_small_file_cache_count | Num | currently cached by BE | |||
doris_be_streaming_load_current_processing | Num | Number of stream load tasks currently running | Contains only tasks sent by the curl command | ||
doris_be_streaming_load_duration_ms | millisecond | The cumulative value of the execution time of all stream load tasks | |||
doris_be_streaming_load_requests_total | Num | value of the number of stream load tasks | Observable task submission frequency via slope | P0 | |
doris_be_stream_load_pipe_count | Num | current stream load data pipelines | Including stream load and routine load tasks | ||
doris_be_stream_load | {type=" load_rows "} | Num | number of rows finally imported by stream load | Including stream load and routine load tasks | P0 |
doris_be_stream_load | {type=" receive_bytes "} | byte | value of the number of bytes received by stream load | Including stream load data received from http , and routine load from kafka read data | P0 |
doris_be_tablet_base_max_compaction_score | Num | The current largest Base Compaction Score | This 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_score | Num | Same as above. Current largest Cumulative Compaction Score | |||
doris_be_tablet_version_num_distribution | Num | The histogram of the number of tablet versions . | Distribution used to reflect the number of tablet versions | P0 | |
doris_be_thrift_connections_total | Num | thrift connections created . like {name="heartbeat"} Indicates the cumulative number of connections to the heartbeat service | This value is the connection of thrift server with BE as the server. | ||
doris_be_thrift_current_connections | Num | current number of thrift connections. like {name="heartbeat"} Indicates the current number of connections to the heartbeat service . | Same as above | ||
doris_be_thrift_opened_clients | Num | thrift clients currently open . like {name="frontend"} Indicates the number of clients accessing the FE service | |||
doris_be_thrift_used_clients | Num | thrift clients currently in use . like {name="frontend"} Indicates the number of clients being used to access the FE service | |||
doris_be_timeout_canceled_fragment_count | Num | Cumulative value of the number of fragment instances canceled due to timeout | This value may be recorded repeatedly. For example, some fragment instances are canceled multiple times | P0 | |
doris_be_stream_load_txn_request | {type="begin"} | Num | value of the number of transactions started by stream load | Including stream load and routine load tasks | |
doris_be_stream_load_txn_request | {type="commit"} | Num | value of the number of transactions successfully executed by stream load | Same as above | |
doris_be_stream_load_txn_request | {type="rollback"} | value of the number of transactions that failed to execute stream load | Same as above | ||
doris_be_unused_rowsets_count | Num | The number of currently abandoned rowsets | These rowsets will be deleted regularly under normal circumstances. | ||
doris_be_upload_fail_count | Num | cumulative value of rowset failed to be uploaded to remote storage | |||
doris_be_upload_rowset_count | Num | cumulative number of rowsets successfully uploaded to remote storage | |||
doris_be_upload_total_byte | byte | value of rowset data successfully uploaded to remote storage | |||
doris_be_load_bytes | byte | Cumulative quantity sent through tablet sink | Observable amount of imported data | P0 | |
doris_be_load_rows | Num | Cumulative number of rows sent through tablet sink | Observable amount of imported data | P0 | |
fragment_thread_pool_queue_size | Num | current query execution thread pool waiting queue | If it is greater than zero, the query thread has been exhausted and queries will pile up. | P0 | |
doris_be_all_rowsets_num | Num | All currently rowset number of | P0 | ||
doris_be_all_segments_num | Num | The number of all current segments | P0 | ||
doris_be_heavy_work_max_threads | Num | brpc Number of heavy thread pool threads | p0 | ||
doris_be_light_work_max_threads | Num | brpc Number of light thread pool threads | p0 | ||
doris_be_heavy_work_pool_queue_size | Num | brpc 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_size | Num | brpc 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_threads | Num | brpc Number of active threads in heavy thread pool | p0 | ||
doris_be_light_work_active_threads | Num | brpc Number of active threads in light thread pool | p0 |
Machine metrics
name | Label | unit | Description | Impact | Grade |
---|---|---|---|---|---|
doris_be_cpu | Num | CPU 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 cpu0 | CPU usage can be calculated | P0 | |
doris_be_disk_bytes_read | byte | The 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_written | byte | The cumulative value of disk writes. The collection method is the same as above | |||
doris_be_disk_io_time_ms | byte | The collection method is the same as above | IO Util can be calculated | P0 | |
doris_be_disk_io_time_weighted | byte | The collection method is the same as above | |||
doris_be_disk_reads_completed | byte | The collection method is the same as above | |||
doris_be_disk_read_time_ms | byte | The collection method is the same as above | |||
doris_be_disk_writes_completed | byte | The collection method is the same as above | |||
doris_be_disk_write_time_ms | byte | The collection method is the same as above | |||
doris_be_fd_num_limit | Num | System file handle limit upper limit. from /proc/sys/fs/file-nr collection | |||
doris_be_fd_num_used | Num | The number of file handles used by the system . from /proc/sys/fs/file-nr collection | |||
doris_be_file_created_total | Num | Cumulative number of local file creation times | all calls local_file_writer And finally close the file count | ||
doris_be_load_average | Num | Machine Load Avg Metric metrics. For example, {mode="15_minutes"} is 15 minutes Load Avg | Observe the overall machine load | P0 | |
doris_be_max_disk_io_util_percent | percentage | value of the disk with the largest IO UTIL among all disks | P0 | ||
doris_be_max_network_receive_bytes_rate | Bytes / second | The maximum receive rate calculated among all network cards | P0 | ||
doris_be_max_network_send_bytes_rate | Bytes / second | The calculated maximum sending rate among all network cards | P0 | ||
doris_be_memory_pgpgin | byte | The amount of data written by the system from disk to memory page | |||
doris_be_memory_pgpgout | byte | The amount of data written to disk by system memory pages | |||
doris_be_memory_pswpin | byte | The number of times the system swapped from disk to memory | Normally, swap should be turned off, so this value should be 0 | ||
doris_be_memory_pswpout | byte | The number of times the system swapped from memory to disk | Normally, swap should be turned off, so this value should be 0 | ||
doris_be_network_receive_bytes | byte | each network card are accumulated. Collected from /proc/net/dev | |||
doris_be_network_receive_packets | Num | each network card is accumulated. Collected from /proc/net/dev | |||
doris_be_network_send_bytes | byte | each network card . Collected from /proc/net/dev | |||
doris_be_network_send_packets | Num | The total number of packets sent by each network card is accumulated. Collected from /proc/net/dev | |||
doris_be_proc | {mode=" ctxt_switch "} | Num | CPU context switches . Collected from /proc/stat | Observe whether there are abnormal context switches | P0 |
doris_be_proc | {mode="interrupt"} | Num | CPU interrupts . Collected from /proc/stat | ||
doris_be_proc | {mode=" procs_blocked "} | Num | The number of processes currently blocked in the system (such as waiting for IO ). Collected from /proc/stat | ||
doris_be_proc | {mode=" procs_running "} | Num | The number of processes currently executing on the system . Collected from /proc/stat | ||
doris_be_snmp_tcp_in_errs | Num | tcp packet reception errors. Collected from /proc/net/ snmp | Observable network errors such as retransmission, packet loss, etc. Need and other snmp metrics used together with | P0 | |
doris_be_snmp_tcp_in_segs | Num | tcp packets sent . Collected from /proc/net/ snmp | |||
doris_be_snmp_tcp_out_segs | Num | tcp packets sent. Collected from /proc/net/ snmp | |||
doris_be_snmp_tcp_retrans_segs | Num | TCP packet retransmissions . Collected from /proc/net/ snmp |