github-actions[bot] commented on code in PR #30875: URL: https://github.com/apache/doris/pull/30875#discussion_r1485999340
########## be/src/util/doris_bvar_metrics.h: ########## @@ -0,0 +1,242 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include <bthread/mutex.h> Review Comment: warning: 'bthread/mutex.h' file not found [clang-diagnostic-error] ```cpp #include <bthread/mutex.h> ^ ``` ########## be/src/util/doris_bvar_metrics.h: ########## @@ -0,0 +1,242 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include <bthread/mutex.h> +#include <bvar/latency_recorder.h> +#include <bvar/reducer.h> +#include <bvar/status.h> +#include <memory> +#include <set> +#include <string> +#include <vector> +#include "util/bvar_metrics.h" +#include "util/system_bvar_metrics.h" +namespace doris { + +class DorisBvarMetrics { +public: + std::shared_ptr<BvarAdderMetric<int64_t>> fragment_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> fragment_request_duration_us; + std::shared_ptr<BvarAdderMetric<int64_t>> query_scan_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> query_scan_rows; + + std::shared_ptr<BvarAdderMetric<int64_t>> push_requests_success_total; + std::shared_ptr<BvarAdderMetric<int64_t>> push_requests_fail_total; + std::shared_ptr<BvarAdderMetric<int64_t>> push_request_duration_us; + std::shared_ptr<BvarAdderMetric<int64_t>> push_request_write_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> push_request_write_rows; + + std::shared_ptr<BvarAdderMetric<int64_t>> create_tablet_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> create_tablet_requests_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> drop_tablet_requests_total; + + std::shared_ptr<BvarAdderMetric<int64_t>> report_all_tablets_requests_skip; + + std::shared_ptr<BvarAdderMetric<int64_t>> schema_change_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> schema_change_requests_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> create_rollup_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> create_rollup_requests_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> storage_migrate_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> storage_migrate_v2_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> storage_migrate_v2_requests_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> delete_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> delete_requests_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> clone_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> clone_requests_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> alter_inverted_index_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> alter_inverted_index_requests_failed; + + std::shared_ptr<BvarAdderMetric<int64_t>> finish_task_requests_total; + std::shared_ptr<BvarAdderMetric<int64_t>> finish_task_requests_failed; + + std::shared_ptr<BvarAdderMetric<int64_t>> base_compaction_request_total; + std::shared_ptr<BvarAdderMetric<int64_t>> base_compaction_request_failed; + std::shared_ptr<BvarAdderMetric<int64_t>> cumulative_compaction_request_total; + std::shared_ptr<BvarAdderMetric<int64_t>> cumulative_compaction_request_failed; + + std::shared_ptr<BvarAdderMetric<int64_t>> base_compaction_deltas_total; + std::shared_ptr<BvarAdderMetric<int64_t>> base_compaction_bytes_total; + std::shared_ptr<BvarAdderMetric<int64_t>> cumulative_compaction_deltas_total; + std::shared_ptr<BvarAdderMetric<int64_t>> cumulative_compaction_bytes_total; + + std::shared_ptr<BvarAdderMetric<int64_t>> publish_task_request_total; + std::shared_ptr<BvarAdderMetric<int64_t>> publish_task_failed_total; + + // Counters for segment_v2 + // ----------------------- + // total number of segments read + std::shared_ptr<BvarAdderMetric<int64_t>> segment_read_total; + // total number of rows in queried segments (before index pruning) + std::shared_ptr<BvarAdderMetric<int64_t>> segment_row_total; + + std::shared_ptr<BvarAdderMetric<int64_t>> stream_load_txn_begin_request_total; + std::shared_ptr<BvarAdderMetric<int64_t>> stream_load_txn_commit_request_total; + std::shared_ptr<BvarAdderMetric<int64_t>> stream_load_txn_rollback_request_total; + std::shared_ptr<BvarAdderMetric<int64_t>> stream_receive_bytes_total; + std::shared_ptr<BvarAdderMetric<int64_t>> stream_load_rows_total; + std::shared_ptr<BvarAdderMetric<int64_t>> load_rows; + std::shared_ptr<BvarAdderMetric<int64_t>> load_bytes; + + std::shared_ptr<BvarAdderMetric<int64_t>> memtable_flush_total; + std::shared_ptr<BvarAdderMetric<int64_t>> memtable_flush_duration_us; + + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pool_bytes_total; + std::shared_ptr<BvarAdderMetric<int64_t>> process_thread_num; + std::shared_ptr<BvarAdderMetric<int64_t>> process_fd_num_used; + std::shared_ptr<BvarAdderMetric<int64_t>> process_fd_num_limit_soft; + std::shared_ptr<BvarAdderMetric<int64_t>> process_fd_num_limit_hard; + + // the max compaction score of all tablets. + // Record base and cumulative scores separately, because + // we need to get the larger of the two. + std::shared_ptr<BvarAdderMetric<int64_t>> tablet_cumulative_max_compaction_score; + std::shared_ptr<BvarAdderMetric<int64_t>> tablet_base_max_compaction_score; + + std::shared_ptr<BvarAdderMetric<int64_t>> all_rowsets_num; + std::shared_ptr<BvarAdderMetric<int64_t>> all_segments_num; + + // permits have been used for all compaction tasks + std::shared_ptr<BvarAdderMetric<int64_t>> compaction_used_permits; + // permits required by the compaction task which is waiting for permits + std::shared_ptr<BvarAdderMetric<int64_t>> compaction_waitting_permits; + + // HistogramMetric* tablet_version_num_distribution; + + // The following metrics will be calculated + // by metric calculator + std::shared_ptr<BvarAdderMetric<int64_t>> query_scan_bytes_per_second; + + // Metrics related with file reader/writer + std::shared_ptr<BvarAdderMetric<int64_t>> local_file_reader_total; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_file_reader_total; + std::shared_ptr<BvarAdderMetric<int64_t>> hdfs_file_reader_total; + std::shared_ptr<BvarAdderMetric<int64_t>> broker_file_reader_total; + std::shared_ptr<BvarAdderMetric<int64_t>> local_file_writer_total; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_file_writer_total; + std::shared_ptr<BvarAdderMetric<int64_t>> file_created_total; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_file_created_total; + std::shared_ptr<BvarAdderMetric<int64_t>> local_bytes_read_total; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_bytes_read_total; + std::shared_ptr<BvarAdderMetric<int64_t>> local_bytes_written_total; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_bytes_written_total; + + std::shared_ptr<BvarAdderMetric<int64_t>> local_file_open_reading; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_file_open_reading; + std::shared_ptr<BvarAdderMetric<int64_t>> hdfs_file_open_reading; + std::shared_ptr<BvarAdderMetric<int64_t>> broker_file_open_reading; + std::shared_ptr<BvarAdderMetric<int64_t>> local_file_open_writing; + std::shared_ptr<BvarAdderMetric<int64_t>> s3_file_open_writing; + + // Size of some global containers + std::shared_ptr<BvarAdderMetric<uint64_t>> rowset_count_generated_and_in_use; + std::shared_ptr<BvarAdderMetric<uint64_t>> unused_rowsets_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> broker_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> data_stream_receiver_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> fragment_endpoint_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> active_scan_context_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> fragment_instance_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> load_channel_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> result_buffer_block_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> result_block_queue_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> routine_load_task_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> small_file_cache_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> stream_load_pipe_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> new_stream_load_pipe_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> brpc_endpoint_stub_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> brpc_function_endpoint_stub_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> tablet_writer_count; + + std::shared_ptr<BvarAdderMetric<uint64_t>> segcompaction_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> compaction_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> load_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> load_channel_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> memtable_memory_limiter_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> query_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> schema_change_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> storage_migration_mem_consumption; + std::shared_ptr<BvarAdderMetric<uint64_t>> tablet_meta_mem_consumption; + + // Cache metrics + std::shared_ptr<BvarAdderMetric<uint64_t>> query_cache_memory_total_byte; + std::shared_ptr<BvarAdderMetric<uint64_t>> query_cache_sql_total_count; + std::shared_ptr<BvarAdderMetric<uint64_t>> query_cache_partition_total_count; + + std::shared_ptr<BvarAdderMetric<int64_t>> lru_cache_memory_bytes; + + std::shared_ptr<BvarAdderMetric<uint64_t>> scanner_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> add_batch_task_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> send_batch_thread_pool_thread_num; + std::shared_ptr<BvarAdderMetric<uint64_t>> send_batch_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> fragment_thread_pool_queue_size; + + // Upload metrics + std::shared_ptr<BvarAdderMetric<uint64_t>> upload_total_byte; + std::shared_ptr<BvarAdderMetric<int64_t>> upload_rowset_count; + std::shared_ptr<BvarAdderMetric<int64_t>> upload_fail_count; + + std::shared_ptr<BvarAdderMetric<uint64_t>> light_work_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> heavy_work_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> heavy_work_active_threads; + std::shared_ptr<BvarAdderMetric<uint64_t>> light_work_active_threads; + + std::shared_ptr<BvarAdderMetric<uint64_t>> heavy_work_pool_max_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> light_work_pool_max_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> heavy_work_max_threads; + std::shared_ptr<BvarAdderMetric<uint64_t>> light_work_max_threads; + + std::shared_ptr<BvarAdderMetric<uint64_t>> flush_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> flush_thread_pool_thread_num; + + std::shared_ptr<BvarAdderMetric<uint64_t>> local_scan_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> local_scan_thread_pool_thread_num; + std::shared_ptr<BvarAdderMetric<uint64_t>> remote_scan_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> remote_scan_thread_pool_thread_num; + std::shared_ptr<BvarAdderMetric<uint64_t>> limited_scan_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> limited_scan_thread_pool_thread_num; + std::shared_ptr<BvarAdderMetric<uint64_t>> group_local_scan_thread_pool_queue_size; + std::shared_ptr<BvarAdderMetric<uint64_t>> group_local_scan_thread_pool_thread_num; + + static DorisBvarMetrics* instance() { + static DorisBvarMetrics metrics; + return &metrics; + } + + void initialize( + bool init_system_metrics = false, + const std::set<std::string>& disk_devices = std::set<std::string>(), + const std::vector<std::string>& network_interfaces = std::vector<std::string>()); + + void register_entity(BvarMetricEntity entity); + SystemBvarMetrics* system_metrics() { return system_metrics_.get(); } + + const std::string to_prometheus(); + +private: + DorisBvarMetrics(); + +private: Review Comment: warning: redundant access specifier has the same accessibility as the previous access specifier [readability-redundant-access-specifiers] ```suggestion ``` <details> <summary>Additional context</summary> **be/src/util/doris_bvar_metrics.h:229:** previously declared here ```cpp private: ^ ``` </details> ########## be/src/util/system_bvar_metrics.cpp: ########## @@ -0,0 +1,1130 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "util/system_bvar_metrics.h" + +#include <ctype.h> +// IWYU pragma: no_include <bthread/errno.h> +#include <errno.h> // IWYU pragma: keep +#include <glog/logging.h> +#include <inttypes.h> +#include <stdio.h> +#include <stdlib.h> +#include <string.h> + +#include <functional> +#include <ostream> +#include <unordered_map> +#include <utility> +#include <vector> + +#include "gutil/strings/split.h" // for string split +#include "gutil/strtoint.h" // for atoi64 +#include "util/mem_info.h" +#include "util/perf_counters.h" + +namespace doris { + +#define DECLARE_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + auto name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, \ + group_name, labels, core); +#define INIT_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, group_name, \ + labels, core); + +#define INIT_DOUBLE_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<double>>(type, unit, #name, description, group_name, \ + labels, core); +// /proc/stat: http://www.linuxhowtos.org/System/procstat.htm +struct CpuBvarMetrics { + CpuBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, std::string cpu_name) { + DECLARE_INT64_BVAR_METRIC(cpu_user, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "user"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_nice, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "nice"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_system, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "system"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_idle, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "idle"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_iowait, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "iowait"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_irq, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "irq"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_soft_irq, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, + "", "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "soft_irq"}}), + false) + DECLARE_INT64_BVAR_METRIC(cpu_steal, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "steal"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_guest, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "guest"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_guest_nice, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, + "", "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "guest_nice"}}), + false) + entity->register_metric("cpu_user", *cpu_user); + entity->register_metric("cpu_nice", *cpu_nice); + entity->register_metric("cpu_system", *cpu_system); + entity->register_metric("cpu_idle", *cpu_idle); + entity->register_metric("cpu_iowait", *cpu_iowait); + entity->register_metric("cpu_irq", *cpu_irq); + entity->register_metric("cpu_soft_irq", *cpu_soft_irq); + entity->register_metric("cpu_steal", *cpu_steal); + entity->register_metric("cpu_guest", *cpu_guest); + entity->register_metric("cpu_guest_nice", *cpu_guest_nice); + + metrics.emplace_back(cpu_user); + metrics.emplace_back(cpu_nice); + metrics.emplace_back(cpu_system); + metrics.emplace_back(cpu_idle); + metrics.emplace_back(cpu_iowait); + metrics.emplace_back(cpu_irq); + metrics.emplace_back(cpu_soft_irq); + metrics.emplace_back(cpu_steal); + metrics.emplace_back(cpu_guest); + metrics.emplace_back(cpu_guest_nice); + } + + static constexpr int cpu_num_metrics = 10; + std::vector<std::shared_ptr<BvarAdderMetric<int64_t>>> metrics; +}; + +struct MemoryBvarMetrics { + MemoryBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(memory_allocated_bytes, BvarMetricType::GAUGE, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pgpgin, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pgpgout, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pswpin, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pswpout, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), false) + entity->register_metric("memory_allocated_bytes", *memory_allocated_bytes); + entity->register_metric("memory_pgpgin", *memory_pgpgin); + entity->register_metric("memory_pgpgout", *memory_pgpgout); + entity->register_metric("memory_pswpin", *memory_pswpin); + entity->register_metric("memory_pswpout", *memory_pswpout); +#ifndef USE_JEMALLOC + INIT_INT64_BVAR_METRIC(memory_tcmalloc_allocated_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_total_thread_cache_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_central_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_transfer_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_thread_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_unmapped_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + entity->register_metric("memory_tcmalloc_allocated_bytes", + *memory_tcmalloc_allocated_bytes); + entity->register_metric("memory_tcmalloc_total_thread_cache_bytes", + *memory_tcmalloc_total_thread_cache_bytes); + entity->register_metric("memory_tcmalloc_central_cache_free_bytes", + *memory_tcmalloc_central_cache_free_bytes); + entity->register_metric("memory_tcmalloc_transfer_cache_free_bytes", + *memory_tcmalloc_transfer_cache_free_bytes); + entity->register_metric("memory_tcmalloc_thread_cache_free_bytes", + *memory_tcmalloc_thread_cache_free_bytes); + entity->register_metric("memory_tcmalloc_pageheap_free_bytes", + *memory_tcmalloc_pageheap_free_bytes); + entity->register_metric("memory_tcmalloc_pageheap_unmapped_bytes", + *memory_tcmalloc_pageheap_unmapped_bytes); +#else + INIT_INT64_BVAR_METRIC(memory_jemalloc_allocated_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_active_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_metadata_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_resident_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_mapped_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_retained_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_tcache_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pactive_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pdirty_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pmuzzy_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_dirty_purged_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_muzzy_purged_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + + entity->register_metric("memory_jemalloc_allocated_bytes", + *memory_jemalloc_allocated_bytes); + entity->register_metric("memory_jemalloc_active_bytes", *memory_jemalloc_active_bytes); + entity->register_metric("memory_jemalloc_metadata_bytes", *memory_jemalloc_metadata_bytes); + entity->register_metric("memory_jemalloc_resident_bytes", *memory_jemalloc_resident_bytes); + entity->register_metric("memory_jemalloc_mapped_bytes", *memory_jemalloc_mapped_bytes); + entity->register_metric("memory_jemalloc_retained_bytes", *memory_jemalloc_retained_bytes); + entity->register_metric("memory_jemalloc_tcache_bytes", *memory_jemalloc_tcache_bytes); + entity->register_metric("memory_jemalloc_pactive_num", *memory_jemalloc_pactive_num); + entity->register_metric("memory_jemalloc_pdirty_num", *memory_jemalloc_pdirty_num); + entity->register_metric("memory_jemalloc_pmuzzy_num", *memory_jemalloc_pmuzzy_num); + entity->register_metric("memory_jemalloc_dirty_purged_num", + *memory_jemalloc_dirty_purged_num); + entity->register_metric("memory_jemalloc_muzzy_purged_num", + *memory_jemalloc_muzzy_purged_num); + +#endif + } + + // MetricEntity* entity = nullptr; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pgpgin; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pgpgout; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pswpin; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pswpout; + +#ifndef USE_JEMALLOC + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_total_thread_cache_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_central_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_transfer_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_thread_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_pageheap_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_pageheap_unmapped_bytes; +#else + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_active_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_metadata_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_resident_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_mapped_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_retained_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_tcache_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pactive_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pdirty_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pmuzzy_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_dirty_purged_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_muzzy_purged_num; +#endif +}; + +struct DiskBvarMetrics { + DiskBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, const std::string& disk_device) { + INIT_INT64_BVAR_METRIC(disk_reads_completed, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_bytes_read, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, "", + "", BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_read_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_writes_completed, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_bytes_written, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels({{"device", disk_device}}), false); + INIT_INT64_BVAR_METRIC(disk_write_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_io_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_io_time_weigthed, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + entity->register_metric("disk_reads_completed", *disk_reads_completed); + entity->register_metric("disk_bytes_read", *disk_bytes_read); + entity->register_metric("disk_read_time_ms", *disk_read_time_ms); + entity->register_metric("disk_writes_completed", *disk_writes_completed); + entity->register_metric("disk_bytes_written", *disk_bytes_written); + entity->register_metric("disk_write_time_ms", *disk_write_time_ms); + entity->register_metric("disk_io_time_ms", *disk_io_time_ms); + entity->register_metric("disk_io_time_weigthed", *disk_io_time_weigthed); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> disk_reads_completed; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_bytes_read; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_read_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_writes_completed; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_bytes_written; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_write_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_io_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_io_time_weigthed; +}; + +struct NetworkBvarMetrics { + NetworkBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, const std::string& interface) { + INIT_INT64_BVAR_METRIC(network_receive_bytes, BvarMetricType::COUNTER, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels({{"device", interface}}), + false); + INIT_INT64_BVAR_METRIC(network_receive_packets, BvarMetricType::COUNTER, + BvarMetricUnit::PACKETS, "", "", BvarMetric::Labels({{"device", interface}}), + false); + INIT_INT64_BVAR_METRIC(network_send_bytes, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels({{"device", interface}}), false); + INIT_INT64_BVAR_METRIC(network_send_packets, BvarMetricType::COUNTER, + BvarMetricUnit::PACKETS, "", "", BvarMetric::Labels({{"device", interface}}), + false); + entity->register_metric("network_receive_bytes", *network_receive_bytes); + entity->register_metric("network_receive_packets", *network_receive_packets); + entity->register_metric("network_send_bytes", *network_send_bytes); + entity->register_metric("network_send_packets", *network_send_packets); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> network_receive_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> network_receive_packets; + std::shared_ptr<BvarAdderMetric<int64_t>> network_send_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> network_send_packets; +}; + +struct FileDescriptorBvarMetrics { + FileDescriptorBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(fd_num_limit, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(fd_num_used, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false); + entity->register_metric("fd_num_limit", *fd_num_limit); + entity->register_metric("fd_num_used", *fd_num_used); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> fd_num_limit; + std::shared_ptr<BvarAdderMetric<int64_t>> fd_num_used; +}; + +// metrics read from /proc/net/snmp +struct SnmpBvarMetrics { + SnmpBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(snmp_tcp_in_errs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "The number of all problematic TCP packets received", "", BvarMetric::Labels(), + false) + INIT_INT64_BVAR_METRIC(snmp_tcp_retrans_segs, BvarMetricType::COUNTER, + BvarMetricUnit::NOUNIT, "All TCP packets retransmitted", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(snmp_tcp_in_segs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "All received TCP packets", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(snmp_tcp_out_segs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "All send TCP packets with RST mark", "", BvarMetric::Labels(), false) + entity->register_metric("snmp_tcp_in_errs", *snmp_tcp_in_errs); + entity->register_metric("snmp_tcp_retrans_segs", *snmp_tcp_retrans_segs); + entity->register_metric("snmp_tcp_in_segs", *snmp_tcp_in_segs); + entity->register_metric("snmp_tcp_out_segs", *snmp_tcp_out_segs); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_in_errs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_retrans_segs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_in_segs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_out_segs; +}; + +struct LoadAverageBvarMetrics { + LoadAverageBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_DOUBLE_BVAR_METRIC(load_average_1_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "1_minutes"}}), false); + INIT_DOUBLE_BVAR_METRIC(load_average_5_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "5_minutes"}}), false); + INIT_DOUBLE_BVAR_METRIC(load_average_15_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "15_minutes"}}), false); + entity->register_metric("load_average_1_minutes", *load_average_1_minutes); + entity->register_metric("load_average_5_minutes", *load_average_5_minutes); + entity->register_metric("load_average_15_minutes", *load_average_15_minutes); + } + + std::shared_ptr<BvarAdderMetric<double>> load_average_1_minutes; + std::shared_ptr<BvarAdderMetric<double>> load_average_5_minutes; + std::shared_ptr<BvarAdderMetric<double>> load_average_15_minutes; +}; + +struct ProcBvarMetrics { + ProcBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(proc_interrupt, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, "", + "proc", BvarMetric::Labels({{"mode", "interrupt"}}), false); + INIT_INT64_BVAR_METRIC(proc_ctxt_switch, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "ctxt_switch"}}), false); + INIT_INT64_BVAR_METRIC(proc_procs_running, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "procs_running"}}), false); + INIT_INT64_BVAR_METRIC(proc_procs_blocked, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "procs_blocked"}}), false); + entity->register_metric("proc_interrupt", *proc_interrupt); + entity->register_metric("proc_ctxt_switch", *proc_ctxt_switch); + entity->register_metric("proc_procs_running", *proc_procs_running); + entity->register_metric("proc_procs_blocked", *proc_procs_blocked); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> proc_interrupt; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_ctxt_switch; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_procs_running; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_procs_blocked; +}; + +const std::string SystemBvarMetrics::to_prometheus(const std::string& registry_name) { Review Comment: warning: return type 'const std::string' (aka 'const basic_string<char>') is 'const'-qualified at the top level, which may reduce code readability without improving const correctness [readability-const-return-type] be/src/util/system_bvar_metrics.h:44: ```diff - const std::string to_prometheus(const std::string& registry_name); + std::string to_prometheus(const std::string& registry_name); ``` ```suggestion std::string SystemBvarMetrics::to_prometheus(const std::string& registry_name) { ``` ########## be/src/util/doris_bvar_metrics.cpp: ########## @@ -0,0 +1,376 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "doris_bvar_metrics.h" + +#include <sstream> + +namespace doris { + +#define INIT_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, group_name, \ + labels, core); + +#define INIT_UINT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<uint64_t>>(type, unit, #name, description, group_name, \ + labels, core); + +#define ENTITY_REGISTER_METRICS(name, type) \ + auto name##_ptr = std::make_shared<BvarMetricEntity>(#name, type); \ + entities_map_[#name].push_back(name##_ptr); \ + name##_ptr->register_metric(#name, *name); + +// #define INIT_DOUBLE_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ +// name = std::make_shared<BvarAdderMetric<double>>(type, unit, #name, description, group_name, labels, core); + +const std::string DorisBvarMetrics::s_registry_name_ = "doris_be"; + +DorisBvarMetrics::DorisBvarMetrics() { + INIT_INT64_BVAR_METRIC(fragment_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "Total fragment requests received.", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(fragment_request_duration_us, BvarMetricType::COUNTER, + BvarMetricUnit::MICROSECONDS, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(query_scan_bytes, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(query_scan_rows, BvarMetricType::COUNTER, BvarMetricUnit::ROWS, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(push_requests_success_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "push_requests_total", + BvarMetric::Labels({{"status", "SUCCESS"}}), false); + INIT_INT64_BVAR_METRIC(push_requests_fail_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "push_requests_total", + BvarMetric::Labels({{"status", "FAIL"}}), false); + INIT_INT64_BVAR_METRIC(push_request_duration_us, BvarMetricType::COUNTER, + BvarMetricUnit::MICROSECONDS, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(push_request_write_bytes, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(push_request_write_rows, BvarMetricType::COUNTER, BvarMetricUnit::ROWS, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(create_tablet_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "create_tablet"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(create_tablet_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "create_tablet"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(drop_tablet_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "drop_tablet"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(report_all_tablets_requests_skip, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "report_all_tablets"}, {"status", "skip"}}), false); + INIT_INT64_BVAR_METRIC(schema_change_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "schema_change"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(schema_change_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "schema_change"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(create_rollup_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "create_rollup"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(create_rollup_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "create_rollup"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(storage_migrate_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "storage_migrate"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(storage_migrate_v2_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "storage_migrate_v2"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(storage_migrate_v2_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "storage_migrate_v2"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(delete_requests_total, BvarMetricType::COUNTER, BvarMetricUnit::REQUESTS, + "", "engine_requests_total", + BvarMetric::Labels({{"type", "delete"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(delete_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "delete"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(clone_requests_total, BvarMetricType::COUNTER, BvarMetricUnit::REQUESTS, + "", "engine_requests_total", + BvarMetric::Labels({{"type", "clone"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(clone_requests_failed, BvarMetricType::COUNTER, BvarMetricUnit::REQUESTS, + "", "engine_requests_total", + BvarMetric::Labels({{"type", "clone"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(alter_inverted_index_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "alter_inverted_index"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(alter_inverted_index_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "alter_inverted_index"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(finish_task_requests_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "finish_task"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(finish_task_requests_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "finish_task"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(base_compaction_request_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "base_compaction"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(base_compaction_request_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "base_compaction"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(cumulative_compaction_request_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "cumulative_compaction"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(cumulative_compaction_request_failed, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "cumulative_compaction"}, {"status", "failed"}}), + false); + INIT_INT64_BVAR_METRIC(base_compaction_deltas_total, BvarMetricType::COUNTER, + BvarMetricUnit::ROWSETS, "", "compaction_deltas_total", + BvarMetric::Labels({{"type", "base"}}), false); + INIT_INT64_BVAR_METRIC(base_compaction_bytes_total, BvarMetricType::COUNTER, + BvarMetricUnit::BYTES, "", "compaction_bytes_total", + BvarMetric::Labels({{"type", "base"}}), false); + INIT_INT64_BVAR_METRIC(cumulative_compaction_deltas_total, BvarMetricType::COUNTER, + BvarMetricUnit::ROWSETS, "", "compaction_deltas_total", + BvarMetric::Labels({{"type", "cumulative"}}), false); + INIT_INT64_BVAR_METRIC(cumulative_compaction_bytes_total, BvarMetricType::COUNTER, + BvarMetricUnit::BYTES, "", "compaction_bytes_total", + BvarMetric::Labels({{"type", "cumulative"}}), false); + INIT_INT64_BVAR_METRIC(publish_task_request_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "publish"}, {"status", "total"}}), false); + INIT_INT64_BVAR_METRIC(publish_task_failed_total, BvarMetricType::COUNTER, + BvarMetricUnit::REQUESTS, "", "engine_requests_total", + BvarMetric::Labels({{"type", "publish"}, {"status", "failed"}}), false); + INIT_INT64_BVAR_METRIC(segment_read_total, BvarMetricType::COUNTER, BvarMetricUnit::OPERATIONS, + "(segment_v2) total number of segments read", "segment_read", + BvarMetric::Labels({{"type", "segment_read_total"}}), false) + INIT_INT64_BVAR_METRIC( + segment_row_total, BvarMetricType::COUNTER, BvarMetricUnit::ROWS, + "(segment_v2) total number of rows in queried segments (before index pruning)", + "segment_read", BvarMetric::Labels({{"type", "segment_row_total"}}), false) + INIT_INT64_BVAR_METRIC(stream_load_txn_begin_request_total, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "stream_load_txn_request", + BvarMetric::Labels({{"type", "begin"}}), false) + INIT_INT64_BVAR_METRIC(stream_load_txn_commit_request_total, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "stream_load_txn_request", + BvarMetric::Labels({{"type", "commit"}}), false) + INIT_INT64_BVAR_METRIC(stream_load_txn_rollback_request_total, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "stream_load_txn_request", + BvarMetric::Labels({{"type", "rollback"}}), false) + INIT_INT64_BVAR_METRIC(stream_receive_bytes_total, BvarMetricType::COUNTER, + BvarMetricUnit::BYTES, "", "stream_load", + BvarMetric::Labels({{"type", "receive_bytes"}}), false) + INIT_INT64_BVAR_METRIC(stream_load_rows_total, BvarMetricType::COUNTER, BvarMetricUnit::ROWS, + "", "stream_load", BvarMetric::Labels({{"type", "load_rows"}}), false) + INIT_INT64_BVAR_METRIC(load_rows, BvarMetricType::COUNTER, BvarMetricUnit::ROWS, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(load_bytes, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memtable_flush_total, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memtable_flush_duration_us, BvarMetricType::COUNTER, + BvarMetricUnit::MICROSECONDS, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pool_bytes_total, BvarMetricType::GAUGE, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(process_thread_num, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(process_fd_num_used, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(process_fd_num_limit_soft, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(process_fd_num_limit_hard, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(tablet_cumulative_max_compaction_score, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(tablet_base_max_compaction_score, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(all_rowsets_num, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(all_segments_num, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(compaction_used_permits, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(compaction_waitting_permits, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "", BvarMetric::Labels(), false) + // INIT_INT64_BVAR_METRIC(tablet_version_num_distribution, BvarMetricType::HISTOGRAM, BvarMetricUnit::NOUNIT, + // "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(query_scan_bytes_per_second, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(local_file_reader_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_file_reader_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(hdfs_file_reader_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(broker_file_reader_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(local_file_writer_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_file_writer_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(file_created_total, BvarMetricType::COUNTER, BvarMetricUnit::FILESYSTEM, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_file_created_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(local_bytes_read_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_bytes_read_total, BvarMetricType::COUNTER, BvarMetricUnit::FILESYSTEM, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(local_bytes_written_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_bytes_written_total, BvarMetricType::COUNTER, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(local_file_open_reading, BvarMetricType::GAUGE, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_file_open_reading, BvarMetricType::GAUGE, BvarMetricUnit::FILESYSTEM, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(hdfs_file_open_reading, BvarMetricType::GAUGE, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(broker_file_open_reading, BvarMetricType::GAUGE, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(local_file_open_writing, BvarMetricType::GAUGE, + BvarMetricUnit::FILESYSTEM, "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(s3_file_open_writing, BvarMetricType::GAUGE, BvarMetricUnit::FILESYSTEM, + "", "", BvarMetric::Labels(), false) + INIT_UINT64_BVAR_METRIC(query_cache_memory_total_byte, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), true) + INIT_UINT64_BVAR_METRIC(query_cache_sql_total_count, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "", BvarMetric::Labels(), true) + INIT_UINT64_BVAR_METRIC(query_cache_partition_total_count, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "", BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(lru_cache_memory_bytes, BvarMetricType::GAUGE, BvarMetricUnit::BYTES, "", + "", BvarMetric::Labels(), true) + INIT_UINT64_BVAR_METRIC(upload_total_byte, BvarMetricType::GAUGE, BvarMetricUnit::BYTES, "", "", + BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(upload_rowset_count, BvarMetricType::COUNTER, BvarMetricUnit::ROWSETS, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(upload_fail_count, BvarMetricType::COUNTER, BvarMetricUnit::ROWSETS, "", + "", BvarMetric::Labels(), false) + + ENTITY_REGISTER_METRICS(fragment_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(fragment_request_duration_us, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(query_scan_bytes, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(query_scan_rows, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(push_requests_success_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(push_requests_fail_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(push_request_duration_us, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(push_request_write_bytes, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(push_request_write_rows, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(create_tablet_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(create_tablet_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(drop_tablet_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(report_all_tablets_requests_skip, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(schema_change_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(schema_change_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(create_rollup_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(create_rollup_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(storage_migrate_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(storage_migrate_v2_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(storage_migrate_v2_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(delete_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(delete_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(clone_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(clone_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(alter_inverted_index_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(alter_inverted_index_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(finish_task_requests_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(finish_task_requests_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(base_compaction_request_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(base_compaction_request_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(cumulative_compaction_request_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(cumulative_compaction_request_failed, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(base_compaction_deltas_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(base_compaction_bytes_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(cumulative_compaction_deltas_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(cumulative_compaction_bytes_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(publish_task_request_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(publish_task_failed_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(segment_read_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(segment_row_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(stream_load_txn_begin_request_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(stream_load_txn_commit_request_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(stream_load_txn_rollback_request_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(stream_receive_bytes_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(stream_load_rows_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(load_rows, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(load_bytes, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(memtable_flush_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(memtable_flush_duration_us, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(memory_pool_bytes_total, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(process_thread_num, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(process_fd_num_used, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(process_fd_num_limit_soft, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(process_fd_num_limit_hard, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(tablet_cumulative_max_compaction_score, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(tablet_base_max_compaction_score, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(all_rowsets_num, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(all_segments_num, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(compaction_used_permits, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(compaction_waitting_permits, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(local_file_reader_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(query_scan_bytes_per_second, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(s3_file_reader_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(hdfs_file_reader_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(broker_file_reader_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(local_file_writer_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(s3_file_writer_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(file_created_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(s3_file_created_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(local_bytes_read_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(s3_bytes_read_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(local_bytes_written_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(s3_bytes_written_total, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(local_file_open_reading, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(s3_file_open_reading, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(hdfs_file_open_reading, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(broker_file_open_reading, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(local_file_open_writing, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(s3_file_open_writing, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(query_cache_memory_total_byte, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(query_cache_sql_total_count, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(query_cache_partition_total_count, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(lru_cache_memory_bytes, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(upload_total_byte, BvarMetricType::GAUGE) + ENTITY_REGISTER_METRICS(upload_rowset_count, BvarMetricType::COUNTER) + ENTITY_REGISTER_METRICS(upload_fail_count, BvarMetricType::COUNTER) +} + +void DorisBvarMetrics::initialize(bool init_system_metrics, + const std::set<std::string>& disk_devices, + const std::vector<std::string>& network_interfaces) { + if (init_system_metrics) { + system_metrics_ = std::make_unique<SystemBvarMetrics>(disk_devices, network_interfaces); + } +} + +void DorisBvarMetrics::register_entity(BvarMetricEntity entity) {} + +const std::string DorisBvarMetrics::to_prometheus() { Review Comment: warning: return type 'const int' is 'const'-qualified at the top level, which may reduce code readability without improving const correctness [readability-const-return-type] ```suggestion std::string DorisBvarMetrics::to_prometheus() { ``` ########## be/src/util/system_bvar_metrics.cpp: ########## @@ -0,0 +1,1130 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "util/system_bvar_metrics.h" + +#include <ctype.h> +// IWYU pragma: no_include <bthread/errno.h> +#include <errno.h> // IWYU pragma: keep +#include <glog/logging.h> +#include <inttypes.h> +#include <stdio.h> +#include <stdlib.h> +#include <string.h> + +#include <functional> +#include <ostream> +#include <unordered_map> +#include <utility> +#include <vector> + +#include "gutil/strings/split.h" // for string split +#include "gutil/strtoint.h" // for atoi64 +#include "util/mem_info.h" +#include "util/perf_counters.h" + +namespace doris { + +#define DECLARE_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + auto name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, \ + group_name, labels, core); +#define INIT_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, group_name, \ + labels, core); + +#define INIT_DOUBLE_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<double>>(type, unit, #name, description, group_name, \ + labels, core); +// /proc/stat: http://www.linuxhowtos.org/System/procstat.htm +struct CpuBvarMetrics { + CpuBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, std::string cpu_name) { + DECLARE_INT64_BVAR_METRIC(cpu_user, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "user"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_nice, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "nice"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_system, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "system"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_idle, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "idle"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_iowait, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "iowait"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_irq, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "irq"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_soft_irq, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, + "", "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "soft_irq"}}), + false) + DECLARE_INT64_BVAR_METRIC(cpu_steal, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "steal"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_guest, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "guest"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_guest_nice, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, + "", "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "guest_nice"}}), + false) + entity->register_metric("cpu_user", *cpu_user); + entity->register_metric("cpu_nice", *cpu_nice); + entity->register_metric("cpu_system", *cpu_system); + entity->register_metric("cpu_idle", *cpu_idle); + entity->register_metric("cpu_iowait", *cpu_iowait); + entity->register_metric("cpu_irq", *cpu_irq); + entity->register_metric("cpu_soft_irq", *cpu_soft_irq); + entity->register_metric("cpu_steal", *cpu_steal); + entity->register_metric("cpu_guest", *cpu_guest); + entity->register_metric("cpu_guest_nice", *cpu_guest_nice); + + metrics.emplace_back(cpu_user); + metrics.emplace_back(cpu_nice); + metrics.emplace_back(cpu_system); + metrics.emplace_back(cpu_idle); + metrics.emplace_back(cpu_iowait); + metrics.emplace_back(cpu_irq); + metrics.emplace_back(cpu_soft_irq); + metrics.emplace_back(cpu_steal); + metrics.emplace_back(cpu_guest); + metrics.emplace_back(cpu_guest_nice); + } + + static constexpr int cpu_num_metrics = 10; + std::vector<std::shared_ptr<BvarAdderMetric<int64_t>>> metrics; +}; + +struct MemoryBvarMetrics { + MemoryBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(memory_allocated_bytes, BvarMetricType::GAUGE, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pgpgin, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pgpgout, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pswpin, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pswpout, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), false) + entity->register_metric("memory_allocated_bytes", *memory_allocated_bytes); + entity->register_metric("memory_pgpgin", *memory_pgpgin); + entity->register_metric("memory_pgpgout", *memory_pgpgout); + entity->register_metric("memory_pswpin", *memory_pswpin); + entity->register_metric("memory_pswpout", *memory_pswpout); +#ifndef USE_JEMALLOC + INIT_INT64_BVAR_METRIC(memory_tcmalloc_allocated_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_total_thread_cache_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_central_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_transfer_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_thread_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_unmapped_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + entity->register_metric("memory_tcmalloc_allocated_bytes", + *memory_tcmalloc_allocated_bytes); + entity->register_metric("memory_tcmalloc_total_thread_cache_bytes", + *memory_tcmalloc_total_thread_cache_bytes); + entity->register_metric("memory_tcmalloc_central_cache_free_bytes", + *memory_tcmalloc_central_cache_free_bytes); + entity->register_metric("memory_tcmalloc_transfer_cache_free_bytes", + *memory_tcmalloc_transfer_cache_free_bytes); + entity->register_metric("memory_tcmalloc_thread_cache_free_bytes", + *memory_tcmalloc_thread_cache_free_bytes); + entity->register_metric("memory_tcmalloc_pageheap_free_bytes", + *memory_tcmalloc_pageheap_free_bytes); + entity->register_metric("memory_tcmalloc_pageheap_unmapped_bytes", + *memory_tcmalloc_pageheap_unmapped_bytes); +#else + INIT_INT64_BVAR_METRIC(memory_jemalloc_allocated_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_active_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_metadata_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_resident_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_mapped_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_retained_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_tcache_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pactive_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pdirty_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pmuzzy_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_dirty_purged_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_muzzy_purged_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + + entity->register_metric("memory_jemalloc_allocated_bytes", + *memory_jemalloc_allocated_bytes); + entity->register_metric("memory_jemalloc_active_bytes", *memory_jemalloc_active_bytes); + entity->register_metric("memory_jemalloc_metadata_bytes", *memory_jemalloc_metadata_bytes); + entity->register_metric("memory_jemalloc_resident_bytes", *memory_jemalloc_resident_bytes); + entity->register_metric("memory_jemalloc_mapped_bytes", *memory_jemalloc_mapped_bytes); + entity->register_metric("memory_jemalloc_retained_bytes", *memory_jemalloc_retained_bytes); + entity->register_metric("memory_jemalloc_tcache_bytes", *memory_jemalloc_tcache_bytes); + entity->register_metric("memory_jemalloc_pactive_num", *memory_jemalloc_pactive_num); + entity->register_metric("memory_jemalloc_pdirty_num", *memory_jemalloc_pdirty_num); + entity->register_metric("memory_jemalloc_pmuzzy_num", *memory_jemalloc_pmuzzy_num); + entity->register_metric("memory_jemalloc_dirty_purged_num", + *memory_jemalloc_dirty_purged_num); + entity->register_metric("memory_jemalloc_muzzy_purged_num", + *memory_jemalloc_muzzy_purged_num); + +#endif + } + + // MetricEntity* entity = nullptr; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pgpgin; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pgpgout; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pswpin; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pswpout; + +#ifndef USE_JEMALLOC + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_total_thread_cache_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_central_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_transfer_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_thread_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_pageheap_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_pageheap_unmapped_bytes; +#else + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_active_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_metadata_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_resident_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_mapped_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_retained_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_tcache_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pactive_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pdirty_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pmuzzy_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_dirty_purged_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_muzzy_purged_num; +#endif +}; + +struct DiskBvarMetrics { + DiskBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, const std::string& disk_device) { + INIT_INT64_BVAR_METRIC(disk_reads_completed, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_bytes_read, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, "", + "", BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_read_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_writes_completed, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_bytes_written, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels({{"device", disk_device}}), false); + INIT_INT64_BVAR_METRIC(disk_write_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_io_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_io_time_weigthed, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + entity->register_metric("disk_reads_completed", *disk_reads_completed); + entity->register_metric("disk_bytes_read", *disk_bytes_read); + entity->register_metric("disk_read_time_ms", *disk_read_time_ms); + entity->register_metric("disk_writes_completed", *disk_writes_completed); + entity->register_metric("disk_bytes_written", *disk_bytes_written); + entity->register_metric("disk_write_time_ms", *disk_write_time_ms); + entity->register_metric("disk_io_time_ms", *disk_io_time_ms); + entity->register_metric("disk_io_time_weigthed", *disk_io_time_weigthed); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> disk_reads_completed; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_bytes_read; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_read_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_writes_completed; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_bytes_written; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_write_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_io_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_io_time_weigthed; +}; + +struct NetworkBvarMetrics { + NetworkBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, const std::string& interface) { + INIT_INT64_BVAR_METRIC(network_receive_bytes, BvarMetricType::COUNTER, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels({{"device", interface}}), + false); + INIT_INT64_BVAR_METRIC(network_receive_packets, BvarMetricType::COUNTER, + BvarMetricUnit::PACKETS, "", "", BvarMetric::Labels({{"device", interface}}), + false); + INIT_INT64_BVAR_METRIC(network_send_bytes, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels({{"device", interface}}), false); + INIT_INT64_BVAR_METRIC(network_send_packets, BvarMetricType::COUNTER, + BvarMetricUnit::PACKETS, "", "", BvarMetric::Labels({{"device", interface}}), + false); + entity->register_metric("network_receive_bytes", *network_receive_bytes); + entity->register_metric("network_receive_packets", *network_receive_packets); + entity->register_metric("network_send_bytes", *network_send_bytes); + entity->register_metric("network_send_packets", *network_send_packets); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> network_receive_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> network_receive_packets; + std::shared_ptr<BvarAdderMetric<int64_t>> network_send_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> network_send_packets; +}; + +struct FileDescriptorBvarMetrics { + FileDescriptorBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(fd_num_limit, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(fd_num_used, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false); + entity->register_metric("fd_num_limit", *fd_num_limit); + entity->register_metric("fd_num_used", *fd_num_used); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> fd_num_limit; + std::shared_ptr<BvarAdderMetric<int64_t>> fd_num_used; +}; + +// metrics read from /proc/net/snmp +struct SnmpBvarMetrics { + SnmpBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(snmp_tcp_in_errs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "The number of all problematic TCP packets received", "", BvarMetric::Labels(), + false) + INIT_INT64_BVAR_METRIC(snmp_tcp_retrans_segs, BvarMetricType::COUNTER, + BvarMetricUnit::NOUNIT, "All TCP packets retransmitted", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(snmp_tcp_in_segs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "All received TCP packets", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(snmp_tcp_out_segs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "All send TCP packets with RST mark", "", BvarMetric::Labels(), false) + entity->register_metric("snmp_tcp_in_errs", *snmp_tcp_in_errs); + entity->register_metric("snmp_tcp_retrans_segs", *snmp_tcp_retrans_segs); + entity->register_metric("snmp_tcp_in_segs", *snmp_tcp_in_segs); + entity->register_metric("snmp_tcp_out_segs", *snmp_tcp_out_segs); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_in_errs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_retrans_segs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_in_segs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_out_segs; +}; + +struct LoadAverageBvarMetrics { + LoadAverageBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_DOUBLE_BVAR_METRIC(load_average_1_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "1_minutes"}}), false); + INIT_DOUBLE_BVAR_METRIC(load_average_5_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "5_minutes"}}), false); + INIT_DOUBLE_BVAR_METRIC(load_average_15_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "15_minutes"}}), false); + entity->register_metric("load_average_1_minutes", *load_average_1_minutes); + entity->register_metric("load_average_5_minutes", *load_average_5_minutes); + entity->register_metric("load_average_15_minutes", *load_average_15_minutes); + } + + std::shared_ptr<BvarAdderMetric<double>> load_average_1_minutes; + std::shared_ptr<BvarAdderMetric<double>> load_average_5_minutes; + std::shared_ptr<BvarAdderMetric<double>> load_average_15_minutes; +}; + +struct ProcBvarMetrics { + ProcBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(proc_interrupt, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, "", + "proc", BvarMetric::Labels({{"mode", "interrupt"}}), false); + INIT_INT64_BVAR_METRIC(proc_ctxt_switch, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "ctxt_switch"}}), false); + INIT_INT64_BVAR_METRIC(proc_procs_running, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "procs_running"}}), false); + INIT_INT64_BVAR_METRIC(proc_procs_blocked, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "procs_blocked"}}), false); + entity->register_metric("proc_interrupt", *proc_interrupt); + entity->register_metric("proc_ctxt_switch", *proc_ctxt_switch); + entity->register_metric("proc_procs_running", *proc_procs_running); + entity->register_metric("proc_procs_blocked", *proc_procs_blocked); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> proc_interrupt; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_ctxt_switch; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_procs_running; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_procs_blocked; +}; + +const std::string SystemBvarMetrics::to_prometheus(const std::string& registry_name) { + std::lock_guard<bthread::Mutex> l(mutex_); + std::stringstream ss; + for (auto& entities : entities_map_) { + if (entities.second.empty()) { + continue; + } + int count = 0; + for (auto& entity : entities.second) { + if (!count) { + ss << "# TYPE " << registry_name << "_" << entity->get_name() << " " + << entity->get_type() << "\n"; + count++; + } + ss << entity->to_prometheus(registry_name); + } + } + return ss.str(); +} + +SystemBvarMetrics::SystemBvarMetrics(const std::set<std::string>& disk_devices, + const std::vector<std::string>& network_interfaces) { + install_cpu_metrics(); + install_memory_metrics(); + install_disk_metrics(disk_devices); + install_net_metrics(network_interfaces); + install_fd_metrics(); + install_snmp_metrics(); + install_load_avg_metrics(); + install_proc_metrics(); + install_max_metrics(); + update(); +} + +SystemBvarMetrics::~SystemBvarMetrics() { + for (auto& it : cpu_metrics_) { + delete it.second; + } + for (auto& it : disk_metrics_) { + delete it.second; + } + for (auto& it : network_metrics_) { + delete it.second; + } + + if (line_ptr_ != nullptr) { + free(line_ptr_); + } +} + +void SystemBvarMetrics::update() { + update_cpu_metrics(); + update_memory_metrics(); + update_disk_metrics(); + update_net_metrics(); + update_fd_metrics(); + update_snmp_metrics(); + update_load_avg_metrics(); + update_proc_metrics(); +} + +void SystemBvarMetrics::install_max_metrics() { + auto max_entity = std::make_shared<BvarMetricEntity>("max", BvarMetricType::GAUGE); + INIT_INT64_BVAR_METRIC(max_disk_io_util_percent, BvarMetricType::GAUGE, BvarMetricUnit::PERCENT, + "", "", BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(max_network_send_bytes_rate, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), true) + INIT_INT64_BVAR_METRIC(max_network_receive_bytes_rate, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), true) + max_entity->register_metric("max_disk_io_util_percent", *max_disk_io_util_percent); + max_entity->register_metric("max_network_send_bytes_rate", *max_network_send_bytes_rate); + max_entity->register_metric("max_network_receive_bytes_rate", *max_network_receive_bytes_rate); + entities_map_["max"].push_back(max_entity); +} + +void SystemBvarMetrics::install_cpu_metrics() { + get_cpu_name(); + for (auto cpu_name : cpu_names_) { + auto cpu_entity = std::make_shared<BvarMetricEntity>("cpu", BvarMetricType::COUNTER); + CpuBvarMetrics* metrics = new CpuBvarMetrics(cpu_entity, cpu_name); + cpu_metrics_.emplace(cpu_name, metrics); + entities_map_["cpu"].push_back(cpu_entity); + } +} + +#ifdef BE_TEST +const char* k_ut_stat_path; +const char* k_ut_diskstats_path; +const char* k_ut_net_dev_path; +const char* k_ut_fd_path; +const char* k_ut_net_snmp_path; +const char* k_ut_load_avg_path; +const char* k_ut_vmstat_path; +#endif + +void SystemBvarMetrics::update_cpu_metrics() { +#ifdef BE_TEST + FILE* fp = fopen(k_ut_stat_path, "r"); +#else + FILE* fp = fopen("/proc/stat", "r"); +#endif + if (fp == nullptr) { + char buf[64]; + LOG(WARNING) << "open /proc/stat failed, errno=" << errno + << ", message=" << strerror_r(errno, buf, 64); + return; + } + + while (getline(&line_ptr_, &line_buf_size_, fp) > 0) { + char cpu[16]; + int64_t values[CpuBvarMetrics::cpu_num_metrics]; + memset(values, 0, sizeof(values)); + int num = sscanf(line_ptr_, + "%15s" + " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 + " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64, + cpu, &values[0], &values[1], &values[2], &values[3], &values[4], + &values[5], &values[6], &values[7], &values[8], &values[9]); + if (num < 4) { + continue; + } + + std::string cpu_name(cpu); + auto it = cpu_metrics_.find(cpu_name); + if (it == cpu_metrics_.end()) { + continue; + } + + for (int i = 0; i < CpuBvarMetrics::cpu_num_metrics; ++i) { + it->second->metrics[i]->set_value(values[i]); + } + } + + if (ferror(fp) != 0) { + char buf[64]; + LOG(WARNING) << "getline failed, errno=" << errno + << ", message=" << strerror_r(errno, buf, 64); + } + + fclose(fp); +} + +void SystemBvarMetrics::install_memory_metrics() { + auto memory_entity = std::make_shared<BvarMetricEntity>("memory", BvarMetricType::GAUGE); + memory_metrics_ = std::make_shared<MemoryBvarMetrics>(memory_entity); + entities_map_["memory"].push_back(memory_entity); +} + +void SystemBvarMetrics::update_memory_metrics() { + memory_metrics_->memory_allocated_bytes->set_value(PerfCounters::get_vm_rss()); + get_metrics_from_proc_vmstat(); +} + +void SystemBvarMetrics::install_disk_metrics(const std::set<std::string>& disk_devices) { + for (auto& disk_device : disk_devices) { + auto disk_entity = std::make_shared<BvarMetricEntity>("disk", BvarMetricType::COUNTER); + DiskBvarMetrics* metrics = new DiskBvarMetrics(disk_entity, disk_device); + entities_map_["disk"].push_back(disk_entity); + disk_metrics_.emplace(disk_device, metrics); + } +} + +void SystemBvarMetrics::update_disk_metrics() { +#ifdef BE_TEST + FILE* fp = fopen(k_ut_diskstats_path, "r"); +#else + FILE* fp = fopen("/proc/diskstats", "r"); +#endif + if (fp == nullptr) { + char buf[64]; + LOG(WARNING) << "open /proc/diskstats failed, errno=" << errno + << ", message=" << strerror_r(errno, buf, 64); + return; + } + + // /proc/diskstats: https://www.kernel.org/doc/Documentation/ABI/testing/procfs-diskstats + // 1 - major number + // 2 - minor mumber + // 3 - device name + // 4 - reads completed successfully + // 5 - reads merged + // 6 - sectors read + // 7 - time spent reading (ms) + // 8 - writes completed + // 9 - writes merged + // 10 - sectors written + // 11 - time spent writing (ms) + // 12 - I/Os currently in progress + // 13 - time spent doing I/Os (ms) + // 14 - weighted time spent doing I/Os (ms) + // I think 1024 is enough for device name + int major = 0; + int minor = 0; + char device[1024]; + int64_t values[11]; + while (getline(&line_ptr_, &line_buf_size_, fp) > 0) { + memset(values, 0, sizeof(values)); + int num = sscanf(line_ptr_, + "%d %d %1023s" + " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 + " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64 " %" PRId64, + &major, &minor, device, &values[0], &values[1], &values[2], &values[3], + &values[4], &values[5], &values[6], &values[7], &values[8], &values[9], + &values[10]); + if (num < 4) { + continue; + } + auto it = disk_metrics_.find(device); + if (it == disk_metrics_.end()) { + continue; + } + // update disk metrics + // reads_completed: 4 reads completed successfully + it->second->disk_reads_completed->set_value(values[0]); + // bytes_read: 6 sectors read * 512; 5 reads merged is ignored + it->second->disk_bytes_read->set_value(values[2] * 512); + // read_time_ms: 7 time spent reading (ms) + it->second->disk_read_time_ms->set_value(values[3]); + // writes_completed: 8 writes completed + it->second->disk_writes_completed->set_value(values[4]); + // bytes_written: 10 sectors write * 512; 9 writes merged is ignored + it->second->disk_bytes_written->set_value(values[6] * 512); + // write_time_ms: 11 time spent writing (ms) + it->second->disk_write_time_ms->set_value(values[7]); + // io_time_ms: 13 time spent doing I/Os (ms) + it->second->disk_io_time_ms->set_value(values[9]); + // io_time_weigthed: 14 - weighted time spent doing I/Os (ms) + it->second->disk_io_time_weigthed->set_value(values[10]); + } + if (ferror(fp) != 0) { + char buf[64]; + LOG(WARNING) << "getline failed, errno=" << errno + << ", message=" << strerror_r(errno, buf, 64); + } + fclose(fp); +} + +void SystemBvarMetrics::install_net_metrics(const std::vector<std::string>& interfaces) { + for (auto& interface : interfaces) { + auto interface_entity = + std::make_shared<BvarMetricEntity>("network", BvarMetricType::COUNTER); + NetworkBvarMetrics* metrics = new NetworkBvarMetrics(interface_entity, interface); + entities_map_["network"].push_back(interface_entity); + network_metrics_.emplace(interface, metrics); + } +} + +void SystemBvarMetrics::update_net_metrics() { Review Comment: warning: function 'update_net_metrics' exceeds recommended size/complexity thresholds [readability-function-size] ```cpp void SystemBvarMetrics::update_net_metrics() { ^ ``` <details> <summary>Additional context</summary> **be/src/util/system_bvar_metrics.cpp:623:** 94 lines including whitespace and comments (threshold 80) ```cpp void SystemBvarMetrics::update_net_metrics() { ^ ``` </details> ########## be/src/util/system_bvar_metrics.cpp: ########## @@ -0,0 +1,1130 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "util/system_bvar_metrics.h" + +#include <ctype.h> +// IWYU pragma: no_include <bthread/errno.h> +#include <errno.h> // IWYU pragma: keep +#include <glog/logging.h> +#include <inttypes.h> +#include <stdio.h> +#include <stdlib.h> +#include <string.h> + +#include <functional> +#include <ostream> +#include <unordered_map> +#include <utility> +#include <vector> + +#include "gutil/strings/split.h" // for string split +#include "gutil/strtoint.h" // for atoi64 +#include "util/mem_info.h" +#include "util/perf_counters.h" + +namespace doris { + +#define DECLARE_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + auto name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, \ + group_name, labels, core); +#define INIT_INT64_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<int64_t>>(type, unit, #name, description, group_name, \ + labels, core); + +#define INIT_DOUBLE_BVAR_METRIC(name, type, unit, description, group_name, labels, core) \ + name = std::make_shared<BvarAdderMetric<double>>(type, unit, #name, description, group_name, \ + labels, core); +// /proc/stat: http://www.linuxhowtos.org/System/procstat.htm +struct CpuBvarMetrics { + CpuBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, std::string cpu_name) { + DECLARE_INT64_BVAR_METRIC(cpu_user, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "user"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_nice, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "nice"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_system, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "system"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_idle, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "idle"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_iowait, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "iowait"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_irq, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "irq"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_soft_irq, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, + "", "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "soft_irq"}}), + false) + DECLARE_INT64_BVAR_METRIC(cpu_steal, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "steal"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_guest, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, "", + "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "guest"}}), false) + DECLARE_INT64_BVAR_METRIC(cpu_guest_nice, BvarMetricType::COUNTER, BvarMetricUnit::PERCENT, + "", "cpu", BvarMetric::Labels({{"device", cpu_name}, {"mode", "guest_nice"}}), + false) + entity->register_metric("cpu_user", *cpu_user); + entity->register_metric("cpu_nice", *cpu_nice); + entity->register_metric("cpu_system", *cpu_system); + entity->register_metric("cpu_idle", *cpu_idle); + entity->register_metric("cpu_iowait", *cpu_iowait); + entity->register_metric("cpu_irq", *cpu_irq); + entity->register_metric("cpu_soft_irq", *cpu_soft_irq); + entity->register_metric("cpu_steal", *cpu_steal); + entity->register_metric("cpu_guest", *cpu_guest); + entity->register_metric("cpu_guest_nice", *cpu_guest_nice); + + metrics.emplace_back(cpu_user); + metrics.emplace_back(cpu_nice); + metrics.emplace_back(cpu_system); + metrics.emplace_back(cpu_idle); + metrics.emplace_back(cpu_iowait); + metrics.emplace_back(cpu_irq); + metrics.emplace_back(cpu_soft_irq); + metrics.emplace_back(cpu_steal); + metrics.emplace_back(cpu_guest); + metrics.emplace_back(cpu_guest_nice); + } + + static constexpr int cpu_num_metrics = 10; + std::vector<std::shared_ptr<BvarAdderMetric<int64_t>>> metrics; +}; + +struct MemoryBvarMetrics { + MemoryBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(memory_allocated_bytes, BvarMetricType::GAUGE, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pgpgin, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pgpgout, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pswpin, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(memory_pswpout, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", + "", BvarMetric::Labels(), false) + entity->register_metric("memory_allocated_bytes", *memory_allocated_bytes); + entity->register_metric("memory_pgpgin", *memory_pgpgin); + entity->register_metric("memory_pgpgout", *memory_pgpgout); + entity->register_metric("memory_pswpin", *memory_pswpin); + entity->register_metric("memory_pswpout", *memory_pswpout); +#ifndef USE_JEMALLOC + INIT_INT64_BVAR_METRIC(memory_tcmalloc_allocated_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_total_thread_cache_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_central_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_transfer_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_thread_cache_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_free_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_unmapped_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + entity->register_metric("memory_tcmalloc_allocated_bytes", + *memory_tcmalloc_allocated_bytes); + entity->register_metric("memory_tcmalloc_total_thread_cache_bytes", + *memory_tcmalloc_total_thread_cache_bytes); + entity->register_metric("memory_tcmalloc_central_cache_free_bytes", + *memory_tcmalloc_central_cache_free_bytes); + entity->register_metric("memory_tcmalloc_transfer_cache_free_bytes", + *memory_tcmalloc_transfer_cache_free_bytes); + entity->register_metric("memory_tcmalloc_thread_cache_free_bytes", + *memory_tcmalloc_thread_cache_free_bytes); + entity->register_metric("memory_tcmalloc_pageheap_free_bytes", + *memory_tcmalloc_pageheap_free_bytes); + entity->register_metric("memory_tcmalloc_pageheap_unmapped_bytes", + *memory_tcmalloc_pageheap_unmapped_bytes); +#else + INIT_INT64_BVAR_METRIC(memory_jemalloc_allocated_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_active_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_metadata_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_resident_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_mapped_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_retained_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_tcache_bytes, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pactive_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pdirty_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_pmuzzy_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_dirty_purged_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(memory_jemalloc_muzzy_purged_num, BvarMetricType::GAUGE, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels(), false); + + entity->register_metric("memory_jemalloc_allocated_bytes", + *memory_jemalloc_allocated_bytes); + entity->register_metric("memory_jemalloc_active_bytes", *memory_jemalloc_active_bytes); + entity->register_metric("memory_jemalloc_metadata_bytes", *memory_jemalloc_metadata_bytes); + entity->register_metric("memory_jemalloc_resident_bytes", *memory_jemalloc_resident_bytes); + entity->register_metric("memory_jemalloc_mapped_bytes", *memory_jemalloc_mapped_bytes); + entity->register_metric("memory_jemalloc_retained_bytes", *memory_jemalloc_retained_bytes); + entity->register_metric("memory_jemalloc_tcache_bytes", *memory_jemalloc_tcache_bytes); + entity->register_metric("memory_jemalloc_pactive_num", *memory_jemalloc_pactive_num); + entity->register_metric("memory_jemalloc_pdirty_num", *memory_jemalloc_pdirty_num); + entity->register_metric("memory_jemalloc_pmuzzy_num", *memory_jemalloc_pmuzzy_num); + entity->register_metric("memory_jemalloc_dirty_purged_num", + *memory_jemalloc_dirty_purged_num); + entity->register_metric("memory_jemalloc_muzzy_purged_num", + *memory_jemalloc_muzzy_purged_num); + +#endif + } + + // MetricEntity* entity = nullptr; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pgpgin; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pgpgout; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pswpin; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_pswpout; + +#ifndef USE_JEMALLOC + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_total_thread_cache_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_central_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_transfer_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_thread_cache_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_pageheap_free_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_tcmalloc_pageheap_unmapped_bytes; +#else + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_allocated_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_active_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_metadata_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_resident_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_mapped_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_retained_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_tcache_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pactive_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pdirty_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_pmuzzy_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_dirty_purged_num; + std::shared_ptr<BvarAdderMetric<int64_t>> memory_jemalloc_muzzy_purged_num; +#endif +}; + +struct DiskBvarMetrics { + DiskBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, const std::string& disk_device) { + INIT_INT64_BVAR_METRIC(disk_reads_completed, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_bytes_read, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, "", + "", BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_read_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_writes_completed, BvarMetricType::COUNTER, + BvarMetricUnit::OPERATIONS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_bytes_written, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels({{"device", disk_device}}), false); + INIT_INT64_BVAR_METRIC(disk_write_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_io_time_ms, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + INIT_INT64_BVAR_METRIC(disk_io_time_weigthed, BvarMetricType::COUNTER, + BvarMetricUnit::MILLISECONDS, "", "", + BvarMetric::Labels({{"device", disk_device}}), false) + entity->register_metric("disk_reads_completed", *disk_reads_completed); + entity->register_metric("disk_bytes_read", *disk_bytes_read); + entity->register_metric("disk_read_time_ms", *disk_read_time_ms); + entity->register_metric("disk_writes_completed", *disk_writes_completed); + entity->register_metric("disk_bytes_written", *disk_bytes_written); + entity->register_metric("disk_write_time_ms", *disk_write_time_ms); + entity->register_metric("disk_io_time_ms", *disk_io_time_ms); + entity->register_metric("disk_io_time_weigthed", *disk_io_time_weigthed); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> disk_reads_completed; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_bytes_read; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_read_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_writes_completed; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_bytes_written; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_write_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_io_time_ms; + std::shared_ptr<BvarAdderMetric<int64_t>> disk_io_time_weigthed; +}; + +struct NetworkBvarMetrics { + NetworkBvarMetrics(std::shared_ptr<BvarMetricEntity> entity, const std::string& interface) { + INIT_INT64_BVAR_METRIC(network_receive_bytes, BvarMetricType::COUNTER, + BvarMetricUnit::BYTES, "", "", BvarMetric::Labels({{"device", interface}}), + false); + INIT_INT64_BVAR_METRIC(network_receive_packets, BvarMetricType::COUNTER, + BvarMetricUnit::PACKETS, "", "", BvarMetric::Labels({{"device", interface}}), + false); + INIT_INT64_BVAR_METRIC(network_send_bytes, BvarMetricType::COUNTER, BvarMetricUnit::BYTES, + "", "", BvarMetric::Labels({{"device", interface}}), false); + INIT_INT64_BVAR_METRIC(network_send_packets, BvarMetricType::COUNTER, + BvarMetricUnit::PACKETS, "", "", BvarMetric::Labels({{"device", interface}}), + false); + entity->register_metric("network_receive_bytes", *network_receive_bytes); + entity->register_metric("network_receive_packets", *network_receive_packets); + entity->register_metric("network_send_bytes", *network_send_bytes); + entity->register_metric("network_send_packets", *network_send_packets); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> network_receive_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> network_receive_packets; + std::shared_ptr<BvarAdderMetric<int64_t>> network_send_bytes; + std::shared_ptr<BvarAdderMetric<int64_t>> network_send_packets; +}; + +struct FileDescriptorBvarMetrics { + FileDescriptorBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(fd_num_limit, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false); + INIT_INT64_BVAR_METRIC(fd_num_used, BvarMetricType::GAUGE, BvarMetricUnit::NOUNIT, "", "", + BvarMetric::Labels(), false); + entity->register_metric("fd_num_limit", *fd_num_limit); + entity->register_metric("fd_num_used", *fd_num_used); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> fd_num_limit; + std::shared_ptr<BvarAdderMetric<int64_t>> fd_num_used; +}; + +// metrics read from /proc/net/snmp +struct SnmpBvarMetrics { + SnmpBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(snmp_tcp_in_errs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "The number of all problematic TCP packets received", "", BvarMetric::Labels(), + false) + INIT_INT64_BVAR_METRIC(snmp_tcp_retrans_segs, BvarMetricType::COUNTER, + BvarMetricUnit::NOUNIT, "All TCP packets retransmitted", "", + BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(snmp_tcp_in_segs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "All received TCP packets", "", BvarMetric::Labels(), false) + INIT_INT64_BVAR_METRIC(snmp_tcp_out_segs, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "All send TCP packets with RST mark", "", BvarMetric::Labels(), false) + entity->register_metric("snmp_tcp_in_errs", *snmp_tcp_in_errs); + entity->register_metric("snmp_tcp_retrans_segs", *snmp_tcp_retrans_segs); + entity->register_metric("snmp_tcp_in_segs", *snmp_tcp_in_segs); + entity->register_metric("snmp_tcp_out_segs", *snmp_tcp_out_segs); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_in_errs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_retrans_segs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_in_segs; + std::shared_ptr<BvarAdderMetric<int64_t>> snmp_tcp_out_segs; +}; + +struct LoadAverageBvarMetrics { + LoadAverageBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_DOUBLE_BVAR_METRIC(load_average_1_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "1_minutes"}}), false); + INIT_DOUBLE_BVAR_METRIC(load_average_5_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "5_minutes"}}), false); + INIT_DOUBLE_BVAR_METRIC(load_average_15_minutes, BvarMetricType::GAUGE, + BvarMetricUnit::NOUNIT, "", "load_average", + BvarMetric::Labels({{"mode", "15_minutes"}}), false); + entity->register_metric("load_average_1_minutes", *load_average_1_minutes); + entity->register_metric("load_average_5_minutes", *load_average_5_minutes); + entity->register_metric("load_average_15_minutes", *load_average_15_minutes); + } + + std::shared_ptr<BvarAdderMetric<double>> load_average_1_minutes; + std::shared_ptr<BvarAdderMetric<double>> load_average_5_minutes; + std::shared_ptr<BvarAdderMetric<double>> load_average_15_minutes; +}; + +struct ProcBvarMetrics { + ProcBvarMetrics(std::shared_ptr<BvarMetricEntity> entity) { + INIT_INT64_BVAR_METRIC(proc_interrupt, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, "", + "proc", BvarMetric::Labels({{"mode", "interrupt"}}), false); + INIT_INT64_BVAR_METRIC(proc_ctxt_switch, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "ctxt_switch"}}), false); + INIT_INT64_BVAR_METRIC(proc_procs_running, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "procs_running"}}), false); + INIT_INT64_BVAR_METRIC(proc_procs_blocked, BvarMetricType::COUNTER, BvarMetricUnit::NOUNIT, + "", "proc", BvarMetric::Labels({{"mode", "procs_blocked"}}), false); + entity->register_metric("proc_interrupt", *proc_interrupt); + entity->register_metric("proc_ctxt_switch", *proc_ctxt_switch); + entity->register_metric("proc_procs_running", *proc_procs_running); + entity->register_metric("proc_procs_blocked", *proc_procs_blocked); + } + + std::shared_ptr<BvarAdderMetric<int64_t>> proc_interrupt; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_ctxt_switch; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_procs_running; + std::shared_ptr<BvarAdderMetric<int64_t>> proc_procs_blocked; +}; + +const std::string SystemBvarMetrics::to_prometheus(const std::string& registry_name) { Review Comment: warning: method 'to_prometheus' can be made static [readability-convert-member-functions-to-static] be/src/util/system_bvar_metrics.h:44: ```diff - const std::string to_prometheus(const std::string& registry_name); + static const std::string to_prometheus(const std::string& registry_name); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org