github-actions[bot] commented on code in PR #30875: URL: https://github.com/apache/doris/pull/30875#discussion_r1492218009
########## be/src/util/bvar_metrics.cpp: ########## @@ -0,0 +1,215 @@ +// 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/bvar_metrics.h" + +namespace doris { + +std::ostream& operator<<(std::ostream& os, BvarMetricType type) { + switch (type) { + case BvarMetricType::COUNTER: + os << "counter"; + break; + case BvarMetricType::GAUGE: + os << "gauge"; + break; + case BvarMetricType::HISTOGRAM: + os << "histogram"; + break; + case BvarMetricType::SUMMARY: + os << "summary"; + break; + case BvarMetricType::UNTYPED: + os << "untyped"; + break; + default: + os << "unknown"; + break; + } + return os; +} + +const char* unit_name(BvarMetricUnit unit) { + switch (unit) { + case BvarMetricUnit::NANOSECONDS: + return "nanoseconds"; + case BvarMetricUnit::MICROSECONDS: + return "microseconds"; + case BvarMetricUnit::MILLISECONDS: + return "milliseconds"; + case BvarMetricUnit::SECONDS: + return "seconds"; + case BvarMetricUnit::BYTES: + return "bytes"; + case BvarMetricUnit::ROWS: + return "rows"; + case BvarMetricUnit::PERCENT: + return "percent"; + case BvarMetricUnit::REQUESTS: + return "requests"; + case BvarMetricUnit::OPERATIONS: + return "operations"; + case BvarMetricUnit::BLOCKS: + return "blocks"; + case BvarMetricUnit::ROWSETS: + return "rowsets"; + case BvarMetricUnit::CONNECTIONS: + return "rowsets"; + default: + return "nounit"; + } +} + + +template <typename T> +T BvarAdderMetric<T>::get_value() const { + return adder_->get_value(); +} + +template <typename T> +void BvarAdderMetric<T>::increment(T value) { + (*adder_) << value; +} + +template <typename T> +void BvarAdderMetric<T>::set_value(T value) { + adder_->reset(); + (*adder_) << value; +} + +template <typename T> +const std::string BvarAdderMetric<T>::to_prometheus(const std::string& registry_name) { + return registry_name + "_" + name_ + label_string() + " " + value_string() + "\n"; +} + +template <typename T> +std::string BvarAdderMetric<T>::to_core_string(const std::string& registry_name) const { + return registry_name + "_" + name_ + " " + "LONG " + value_string() + "\n"; +} + +template <typename T> +std::string BvarAdderMetric<T>::label_string() const { + if (labels_.empty()) { + return ""; + } + + std::stringstream ss; + ss << "{"; + int i = 0; + for (auto label : labels_) { + if (i++ > 0) { + ss << ","; + } + ss << label.first << "=\"" << label.second << "\""; + } + ss << "}"; + return ss.str(); +} + +template <typename T> +std::string BvarAdderMetric<T>::value_string() const { + return std::to_string(adder_->get_value()); +} + +template <typename T> +void BvarMetricEntity::register_metric(const std::string& name, T metric) { + { + std::lock_guard<bthread::Mutex> l(mutex_); + auto it = metrics_.find(name); + if (it == metrics_.end()) { + metrics_[name] = std::make_shared<T>(metric); + } + } +} + +// void BvarMetricEntity::deregister_metric(const std::string& name) { +// { +// std::lock_guard<bthread::Mutex> l(mutex_); +// auto it = metrics_.find(name); +// if (it != metrics_.end()) { +// metrics_.erase(it); +// } +// } +// } + +// std::shared_ptr<BvarMetric> BvarMetricEntity::get_metric(const std::string& name) { +// { +// std::lock_guard<bthread::Mutex> l(mutex_); +// auto it = metrics_.find(name); +// if (it == metrics_.end()) { +// return nullptr; +// } +// return it->second; +// } +// } + +void BvarMetricEntity::register_hook(const std::string& name, const std::function<void()>& hook) { + std::lock_guard<bthread::Mutex> l(mutex_); +#ifndef BE_TEST + DCHECK(hooks_.find(name) == hooks_.end()) << "hook is already exist! " << entity_name_ << ":" << name; +#endif + hooks_.emplace(name, hook); +} + +void BvarMetricEntity::deregister_hook(const std::string& name) { + std::lock_guard<bthread::Mutex> l(mutex_); + hooks_.erase(name); +} + +void BvarMetricEntity::trigger_hook_unlocked(bool force) const { Review Comment: warning: method 'trigger_hook_unlocked' can be made static [readability-convert-member-functions-to-static] be/src/util/bvar_metrics.h:156: ```diff - void trigger_hook_unlocked(bool force) const; + static void trigger_hook_unlocked(bool force) ; ``` ```suggestion void BvarMetricEntity::trigger_hook_unlocked(bool force) { ``` ########## be/src/util/doris_bvar_metrics.h: ########## @@ -0,0 +1,245 @@ +// 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(); } + + void trigger_all_hooks(bool force); + + const std::string to_prometheus(); + const std::string to_core_string(); + const std::string to_json(bool with_tablet_metrics); +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:232:** previously declared here ```cpp private: ^ ``` </details> ########## be/src/util/system_bvar_metrics.h: ########## @@ -0,0 +1,129 @@ +// 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 <memory> +#include <string> +#include <unordered_map> +#include <vector> + +#include "util/bvar_metrics.h" + +namespace doris { + +struct CpuBvarMetrics; +struct MemoryBvarMetrics; +struct DiskBvarMetrics; +struct NetworkBvarMetrics; +struct FileDescriptorBvarMetrics; +struct SnmpBvarMetrics; +struct LoadAverageBvarMetrics; +struct ProcBvarMetrics; + +class SystemBvarMetrics { +public: + SystemBvarMetrics(const std::set<std::string>& disk_devices, + const std::vector<std::string>& network_interfaces); + + ~SystemBvarMetrics(); + + const std::string to_prometheus(const std::string& registry_name); + const std::string to_core_string(const std::string& registry_name); + void to_json(rj::Document& doc, bool with_tablet_metrics); + // update metrics + void update(); + + void get_disks_io_time(std::map<std::string, int64_t>* map); + int64_t get_max_io_util(const std::map<std::string, int64_t>& lst_value, int64_t interval_sec); + + void get_network_traffic(std::map<std::string, int64_t>* send_map, + std::map<std::string, int64_t>* rcv_map); + void get_max_net_traffic(const std::map<std::string, int64_t>& lst_send_map, + const std::map<std::string, int64_t>& lst_rcv_map, + int64_t interval_sec, int64_t* send_rate, int64_t* rcv_rate); + + void update_max_disk_io_util_percent(const std::map<std::string, int64_t>& lst_value, + int64_t interval_sec); + void update_max_network_send_bytes_rate(int64_t max_send_bytes_rate); + void update_max_network_receive_bytes_rate(int64_t max_receive_bytes_rate); + void update_allocator_metrics(); + + //for UT + CpuBvarMetrics* cpu_metrics(const std::string& name) { return cpu_metrics_[name]; } + MemoryBvarMetrics* memory_metrics() { return memory_metrics_.get(); } + DiskBvarMetrics* disk_metrics(const std::string& name) { return disk_metrics_[name]; } + NetworkBvarMetrics* network_metrics(const std::string& name) { return network_metrics_[name]; } + FileDescriptorBvarMetrics* fd_metrics() { return fd_metrics_.get(); } + SnmpBvarMetrics* snmp_metrics() { return snmp_metrics_.get(); } + LoadAverageBvarMetrics* load_average_metrics() { return load_average_metrics_.get(); } + ProcBvarMetrics* proc_metrics() { return proc_metrics_.get(); } + +private: + void install_cpu_metrics(); + // On Intel(R) Xeon(R) CPU E5-2450 0 @ 2.10GHz; + // read /proc/stat would cost about 170us + void update_cpu_metrics(); + + void install_memory_metrics(); + void update_memory_metrics(); + + void install_disk_metrics(const std::set<std::string>& disk_devices); + void update_disk_metrics(); + + void install_net_metrics(const std::vector<std::string>& interfaces); + void update_net_metrics(); + + void install_fd_metrics(); + void update_fd_metrics(); + + void install_snmp_metrics(); + void update_snmp_metrics(); + + void install_load_avg_metrics(); + void update_load_avg_metrics(); + + void install_proc_metrics(); + void update_proc_metrics(); + + void get_metrics_from_proc_vmstat(); + void get_cpu_name(); + + void install_max_metrics(); + + +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/system_bvar_metrics.h:75:** previously declared here ```cpp private: ^ ``` </details> ########## be/src/util/doris_bvar_metrics.cpp: ########## @@ -0,0 +1,447 @@ +// 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) {} + +void DorisBvarMetrics::trigger_all_hooks(bool force) { Review Comment: warning: method 'trigger_all_hooks' can be made static [readability-convert-member-functions-to-static] be/src/util/doris_bvar_metrics.h:227: ```diff - void trigger_all_hooks(bool force); + static void trigger_all_hooks(bool force); ``` ########## be/src/util/system_bvar_metrics.cpp: ########## @@ -0,0 +1,1183 @@ +// 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) { + entity->trigger_hook_unlocked(false); + if (!count) { + ss << "# TYPE " << registry_name << "_" << entity->entity_name_ << " " + << entity->metrics_type_ << "\n"; + count++; + } + ss << entity->to_prometheus(registry_name); + } + } + return ss.str(); +} + +const std::string SystemBvarMetrics::to_core_string(const std::string& registry_name) { + std::stringstream ss; + ss << max_disk_io_util_percent->to_core_string(registry_name); + ss << max_network_send_bytes_rate->to_core_string(registry_name); + ss << max_network_receive_bytes_rate->to_core_string(registry_name); + return ss.str(); +} + +void SystemBvarMetrics::to_json(rj::Document& doc, bool with_tablet_metrics) { + rj::Document::AllocatorType& allocator = doc.GetAllocator(); + std::lock_guard<bthread::Mutex> l(mutex_); + for (const auto& entities : entities_map_) { + for(const auto& entity : entities.second) { + // if (entity.first->_type == MetricEntityType::kTablet && !with_tablet_metrics) { + // continue; + // } + entity->trigger_hook_unlocked(false); + for (const auto& metric : entity->metrics_) { + rj::Value metric_obj(rj::kObjectType); + // tags + rj::Value tag_obj(rj::kObjectType); + tag_obj.AddMember("metric", rj::Value(metric.second->name_.c_str(), allocator), + allocator); + // MetricPrototype's labels + for (auto& label : metric.second->labels_) { + tag_obj.AddMember(rj::Value(label.first.c_str(), allocator), + rj::Value(label.second.c_str(), allocator), allocator); + } + // MetricEntity's labels + // for (auto& label : entity->labels_) { + // tag_obj.AddMember(rj::Value(label.first.c_str(), allocator), + // rj::Value(label.second.c_str(), allocator), allocator); + // } + metric_obj.AddMember("tags", tag_obj, allocator); + // unit + rj::Value unit_val(unit_name(metric.second->unit_), allocator); + metric_obj.AddMember("unit", unit_val, allocator); + // value + metric_obj.AddMember("value", metric.second->to_json_value(allocator), allocator); + doc.PushBack(metric_obj, allocator); + } + } + } +} + + +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; + } +} + +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; + } + char* line_ptr = nullptr; + size_t line_buf_size = 0; + 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]; + char* line_ptr = nullptr; + size_t line_buf_size = 0; + 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:668:** 95 lines including whitespace and comments (threshold 80) ```cpp void SystemBvarMetrics::update_net_metrics() { ^ ``` </details> -- 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