gavinchou commented on code in PR #30875:
URL: https://github.com/apache/doris/pull/30875#discussion_r1478416749


##########
be/src/olap/task/engine_batch_load_task.cpp:
##########
@@ -274,12 +275,18 @@ Status EngineBatchLoadTask::_push(const TPushReq& request,
 
     if (tablet_info_vec == nullptr) {
         DorisMetrics::instance()->push_requests_fail_total->increment(1);
+        if (config::enable_bvar_metrics) {

Review Comment:
   Why there is a check `if (config::enable_bvar_metrics)` and when to check 
the config (most of 
   references of `DorisBvarMetrics::instance()` do not check it)?



##########
be/src/clucene:
##########


Review Comment:
   It seems this module should not be updated in this PR, is it added by 
mistake?



##########
be/src/service/http_service.cpp:
##########
@@ -164,6 +165,13 @@ Status HttpService::start() {
         _ev_http_server->register_handler(HttpMethod::GET, "/metrics", action);
     }
 
+    // register bvar_metrics
+    {
+        auto action = _pool.add(
+                new BvarMetricsAction(_env, TPrivilegeHier::GLOBAL, 
TPrivilegeType::NONE));

Review Comment:
   Why not just use the brpc built-in bvar service 
`${be_host}:${brpc_port}/brpc_metrics`?



##########
be/src/util/doris_bvar_metrics.h:
##########
@@ -0,0 +1,237 @@
+// 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 <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;

Review Comment:
   These declarations may be verbose/redundant if you want to use a 
container(`metrics_`) and MACRO `INIT_INT64_BVAR_METRIC` to initialize them. We 
have to write bvar name 3 times when defining a bvar such as 
`fragment_requests_total`.  It seems that a single MACRO can do all the work. 
   
   And it may be dangerous for someone to declare the bvar **without using the 
MACRO to initiate it in .cpp file**. It compiles but will cause a coredump at 
runtime.



##########
be/src/util/system_bvar_metrics.h:
##########
@@ -0,0 +1,127 @@
+// 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();
+
+    std::string to_prometheus(const std::string& registry_name) const;
+
+    // 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:
+    std::map<std::string, CpuBvarMetrics*> cpu_metrics_;
+    std::shared_ptr<MemoryBvarMetrics> memory_metrics_;
+    std::map<std::string, DiskBvarMetrics*> disk_metrics_;
+    std::map<std::string, NetworkBvarMetrics*> network_metrics_;
+    std::shared_ptr<FileDescriptorBvarMetrics> fd_metrics_;
+    std::shared_ptr<SnmpBvarMetrics> snmp_metrics_;
+    std::shared_ptr<LoadAverageBvarMetrics> load_average_metrics_;
+    std::shared_ptr<ProcBvarMetrics> proc_metrics_;
+
+    std::vector<std::string> cpu_names_;
+    int proc_net_dev_version_ = 0;
+    char* line_ptr_ = nullptr;
+    size_t line_buf_size_ = 0;

Review Comment:
   are they (`line_ptr_` and `line_buf_size_`) necessary as member vars?
    



##########
be/src/util/bvar_metrics.h:
##########
@@ -0,0 +1,143 @@
+// 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 <map>
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+namespace doris {
+
+enum class BvarMetricType { COUNTER, GAUGE, HISTOGRAM, SUMMARY, UNTYPED };
+enum class BvarMetricUnit {
+    NANOSECONDS,
+    MICROSECONDS,
+    MILLISECONDS,
+    SECONDS,
+    BYTES,
+    ROWS,
+    PERCENT,
+    REQUESTS,
+    OPERATIONS,
+    BLOCKS,
+    ROWSETS,
+    CONNECTIONS,
+    PACKETS,
+    NOUNIT,
+    FILESYSTEM
+};
+
+std::ostream& operator<<(std::ostream& os, BvarMetricType type);
+// const char* unit_name(BvarMetricUnit unit);
+
+using Labels = std::unordered_map<std::string, std::string>;
+
+class BvarMetric {
+public:
+    BvarMetric() = default;
+    virtual ~BvarMetric() = default;
+    BvarMetric(BvarMetric&) = default;
+    BvarMetric(BvarMetricType type, BvarMetricUnit unit, std::string name,
+               std::string description = "", std::string group_name = "", 
Labels labels = Labels(),
+               bool is_core_metric = false)
+            : is_core_metric_(is_core_metric),
+              type_(type),
+              unit_(unit),
+              group_name_(group_name),
+              name_(name),
+              description_(description),
+              labels_(labels) {}
+    virtual std::string to_prometheus(const std::string& registry_name) const 
= 0;
+    // std::string to_json(bool with_tablet_metrics = false) const;
+    // std::string to_core_string() const;
+protected:
+    bool is_core_metric_;
+
+    BvarMetricType type_;
+    BvarMetricUnit unit_;
+
+    // use for expose
+    std::string group_name_; // prefix
+    std::string name_;
+    std::string description_;
+
+    Labels labels_;
+};
+
+// bvar::Adder which support the operation of commutative and associative laws
+template <typename T>
+class BvarAdderMetric : public BvarMetric {
+public:
+    BvarAdderMetric(BvarMetricType type, BvarMetricUnit unit, std::string name,
+                    std::string description = "", std::string group_name = "",
+                    Labels labels = Labels(), bool is_core_metric = false)
+            : BvarMetric(type, unit, name, description, group_name, labels, 
is_core_metric) {
+        // addr::expose_as
+        adder_ = std::make_shared<bvar::Adder<T>>(group_name, name + '_' + 
description);

Review Comment:
   It seems not right to put a `description` in the bvar name which will be 
exposed by brpc metrics by default, 
   the given description may be a looooooooooooooooooooooooooog string with 
white spaces and line breaks.
   
   And, do we need to hide/expose the bvar intentionally?



##########
be/src/util/bvar_metrics.cpp:
##########
@@ -0,0 +1,142 @@
+// 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;
+}
+
+template <typename T>
+T BvarAdderMetric<T>::get_value() {
+    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>
+std::string BvarAdderMetric<T>::to_prometheus(const std::string& 
registry_name) const {
+    return registry_name + "_" + name_ + label_string() + " " + 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);

Review Comment:
   `erase` does not really deregister the bvar from brpc service.



##########
be/src/util/bvar_metrics.h:
##########
@@ -0,0 +1,143 @@
+// 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 <map>
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+namespace doris {
+
+enum class BvarMetricType { COUNTER, GAUGE, HISTOGRAM, SUMMARY, UNTYPED };
+enum class BvarMetricUnit {
+    NANOSECONDS,
+    MICROSECONDS,
+    MILLISECONDS,
+    SECONDS,
+    BYTES,
+    ROWS,
+    PERCENT,
+    REQUESTS,
+    OPERATIONS,
+    BLOCKS,
+    ROWSETS,
+    CONNECTIONS,
+    PACKETS,
+    NOUNIT,
+    FILESYSTEM
+};
+
+std::ostream& operator<<(std::ostream& os, BvarMetricType type);
+// const char* unit_name(BvarMetricUnit unit);
+
+using Labels = std::unordered_map<std::string, std::string>;

Review Comment:
   The scope of `using Labels` is too large, better to keep its original long 
name, or "hide" it into `BvarMetric` as a public type member.



##########
be/src/util/doris_bvar_metrics.cpp:
##########
@@ -0,0 +1,375 @@
+// 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.", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(fragment_request_duration_us, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::MICROSECONDS, "", "", Labels(), 
false)
+    INIT_INT64_BVAR_METRIC(query_scan_bytes, BvarMetricType::COUNTER, 
BvarMetricUnit::BYTES, "", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(query_scan_rows, BvarMetricType::COUNTER, 
BvarMetricUnit::ROWS, "", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(push_requests_success_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", "push_requests_total",
+                           Labels({{"status", "SUCCESS"}}), false);
+    INIT_INT64_BVAR_METRIC(push_requests_fail_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", "push_requests_total",
+                           Labels({{"status", "FAIL"}}), false);
+    INIT_INT64_BVAR_METRIC(push_request_duration_us, BvarMetricType::COUNTER,
+                           BvarMetricUnit::MICROSECONDS, "", "", Labels(), 
false)
+    INIT_INT64_BVAR_METRIC(push_request_write_bytes, BvarMetricType::COUNTER, 
BvarMetricUnit::BYTES,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(push_request_write_rows, BvarMetricType::COUNTER, 
BvarMetricUnit::ROWS,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(create_tablet_requests_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "create_tablet"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(create_tablet_requests_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "create_tablet"}, {"status", 
"failed"}}), false);
+    INIT_INT64_BVAR_METRIC(drop_tablet_requests_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "drop_tablet"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(report_all_tablets_requests_skip, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "report_all_tablets"}, {"status", 
"skip"}}), false);
+    INIT_INT64_BVAR_METRIC(schema_change_requests_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "schema_change"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(schema_change_requests_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "schema_change"}, {"status", 
"failed"}}), false);
+    INIT_INT64_BVAR_METRIC(create_rollup_requests_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "create_rollup"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(create_rollup_requests_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "create_rollup"}, {"status", 
"failed"}}), false);
+    INIT_INT64_BVAR_METRIC(storage_migrate_requests_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "storage_migrate"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(storage_migrate_v2_requests_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "storage_migrate_v2"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(storage_migrate_v2_requests_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "storage_migrate_v2"}, {"status", 
"failed"}}), false);
+    INIT_INT64_BVAR_METRIC(delete_requests_total, BvarMetricType::COUNTER, 
BvarMetricUnit::REQUESTS,
+                           "", "engine_requests_total",
+                           Labels({{"type", "delete"}, {"status", "total"}}), 
false);
+    INIT_INT64_BVAR_METRIC(delete_requests_failed, BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "delete"}, {"status", "failed"}}), 
false);
+    INIT_INT64_BVAR_METRIC(clone_requests_total, BvarMetricType::COUNTER, 
BvarMetricUnit::REQUESTS,
+                           "", "engine_requests_total",
+                           Labels({{"type", "clone"}, {"status", "total"}}), 
false);
+    INIT_INT64_BVAR_METRIC(clone_requests_failed, BvarMetricType::COUNTER, 
BvarMetricUnit::REQUESTS,
+                           "", "engine_requests_total",
+                           Labels({{"type", "clone"}, {"status", "failed"}}), 
false);
+    INIT_INT64_BVAR_METRIC(alter_inverted_index_requests_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "alter_inverted_index"}, 
{"status", "total"}}), false);
+    INIT_INT64_BVAR_METRIC(alter_inverted_index_requests_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "alter_inverted_index"}, 
{"status", "failed"}}), false);
+    INIT_INT64_BVAR_METRIC(finish_task_requests_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "finish_task"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(finish_task_requests_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "finish_task"}, {"status", 
"failed"}}), false);
+    INIT_INT64_BVAR_METRIC(base_compaction_request_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "base_compaction"}, {"status", 
"total"}}), false);
+    INIT_INT64_BVAR_METRIC(base_compaction_request_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "base_compaction"}, {"status", 
"failed"}}), false);
+    INIT_INT64_BVAR_METRIC(cumulative_compaction_request_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "cumulative_compaction"}, 
{"status", "total"}}), false);
+    INIT_INT64_BVAR_METRIC(cumulative_compaction_request_failed, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "cumulative_compaction"}, 
{"status", "failed"}}),
+                           false);
+    INIT_INT64_BVAR_METRIC(base_compaction_deltas_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::ROWSETS, "", 
"compaction_deltas_total",
+                           Labels({{"type", "base"}}), false);
+    INIT_INT64_BVAR_METRIC(base_compaction_bytes_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::BYTES, "", "compaction_bytes_total",
+                           Labels({{"type", "base"}}), false);
+    INIT_INT64_BVAR_METRIC(cumulative_compaction_deltas_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::ROWSETS, "", 
"compaction_deltas_total",
+                           Labels({{"type", "cumulative"}}), false);
+    INIT_INT64_BVAR_METRIC(cumulative_compaction_bytes_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::BYTES, "", "compaction_bytes_total",
+                           Labels({{"type", "cumulative"}}), false);
+    INIT_INT64_BVAR_METRIC(publish_task_request_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           Labels({{"type", "publish"}, {"status", "total"}}), 
false);
+    INIT_INT64_BVAR_METRIC(publish_task_failed_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::REQUESTS, "", 
"engine_requests_total",
+                           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",
+                           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", Labels({{"type", "segment_row_total"}}), false)
+    INIT_INT64_BVAR_METRIC(stream_load_txn_begin_request_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::OPERATIONS, "", 
"stream_load_txn_request",
+                           Labels({{"type", "begin"}}), false)
+    INIT_INT64_BVAR_METRIC(stream_load_txn_commit_request_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::OPERATIONS, "", 
"stream_load_txn_request",
+                           Labels({{"type", "commit"}}), false)
+    INIT_INT64_BVAR_METRIC(stream_load_txn_rollback_request_total, 
BvarMetricType::COUNTER,
+                           BvarMetricUnit::OPERATIONS, "", 
"stream_load_txn_request",
+                           Labels({{"type", "rollback"}}), false)
+    INIT_INT64_BVAR_METRIC(stream_receive_bytes_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::BYTES, "", "stream_load",
+                           Labels({{"type", "receive_bytes"}}), false)
+    INIT_INT64_BVAR_METRIC(stream_load_rows_total, BvarMetricType::COUNTER, 
BvarMetricUnit::ROWS,
+                           "", "stream_load", Labels({{"type", "load_rows"}}), 
false)
+    INIT_INT64_BVAR_METRIC(load_rows, BvarMetricType::COUNTER, 
BvarMetricUnit::ROWS, "", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(load_bytes, BvarMetricType::COUNTER, 
BvarMetricUnit::BYTES, "", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(memtable_flush_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::OPERATIONS, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(memtable_flush_duration_us, BvarMetricType::COUNTER,
+                           BvarMetricUnit::MICROSECONDS, "", "", Labels(), 
false)
+    INIT_INT64_BVAR_METRIC(memory_pool_bytes_total, BvarMetricType::GAUGE, 
BvarMetricUnit::BYTES,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(process_thread_num, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "",
+                           "", Labels(), true)
+    INIT_INT64_BVAR_METRIC(process_fd_num_used, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "",
+                           "", Labels(), true)
+    INIT_INT64_BVAR_METRIC(process_fd_num_limit_soft, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(process_fd_num_limit_hard, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(tablet_cumulative_max_compaction_score, 
BvarMetricType::GAUGE,
+                           BvarMetricUnit::NOUNIT, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(tablet_base_max_compaction_score, 
BvarMetricType::GAUGE,
+                           BvarMetricUnit::NOUNIT, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(all_rowsets_num, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(all_segments_num, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "", "",
+                           Labels(), false)
+    INIT_INT64_BVAR_METRIC(compaction_used_permits, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(compaction_waitting_permits, BvarMetricType::GAUGE,
+                           BvarMetricUnit::NOUNIT, "", "", Labels(), false)
+    // INIT_INT64_BVAR_METRIC(tablet_version_num_distribution, 
BvarMetricType::HISTOGRAM, BvarMetricUnit::NOUNIT,
+    //                        "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(query_scan_bytes_per_second, BvarMetricType::GAUGE,
+                           BvarMetricUnit::BYTES, "", "", Labels(), true)
+    INIT_INT64_BVAR_METRIC(local_file_reader_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_file_reader_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(hdfs_file_reader_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(broker_file_reader_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(local_file_writer_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_file_writer_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(file_created_total, BvarMetricType::COUNTER, 
BvarMetricUnit::FILESYSTEM,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_file_created_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(local_bytes_read_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_bytes_read_total, BvarMetricType::COUNTER, 
BvarMetricUnit::FILESYSTEM,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(local_bytes_written_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_bytes_written_total, BvarMetricType::COUNTER,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(local_file_open_reading, BvarMetricType::GAUGE,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_file_open_reading, BvarMetricType::GAUGE, 
BvarMetricUnit::FILESYSTEM,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(hdfs_file_open_reading, BvarMetricType::GAUGE,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(broker_file_open_reading, BvarMetricType::GAUGE,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(local_file_open_writing, BvarMetricType::GAUGE,
+                           BvarMetricUnit::FILESYSTEM, "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(s3_file_open_writing, BvarMetricType::GAUGE, 
BvarMetricUnit::FILESYSTEM,
+                           "", "", Labels(), false)
+    INIT_UINT64_BVAR_METRIC(query_cache_memory_total_byte, 
BvarMetricType::GAUGE,
+                            BvarMetricUnit::BYTES, "", "", Labels(), true)
+    INIT_UINT64_BVAR_METRIC(query_cache_sql_total_count, BvarMetricType::GAUGE,
+                            BvarMetricUnit::NOUNIT, "", "", Labels(), true)
+    INIT_UINT64_BVAR_METRIC(query_cache_partition_total_count, 
BvarMetricType::GAUGE,
+                            BvarMetricUnit::NOUNIT, "", "", Labels(), true)
+    INIT_INT64_BVAR_METRIC(lru_cache_memory_bytes, BvarMetricType::GAUGE, 
BvarMetricUnit::BYTES, "",
+                           "", Labels(), true)
+    INIT_UINT64_BVAR_METRIC(upload_total_byte, BvarMetricType::GAUGE, 
BvarMetricUnit::BYTES, "", "",
+                            Labels(), true)
+    INIT_INT64_BVAR_METRIC(upload_rowset_count, BvarMetricType::COUNTER, 
BvarMetricUnit::ROWSETS,
+                           "", "", Labels(), false)
+    INIT_INT64_BVAR_METRIC(upload_fail_count, BvarMetricType::COUNTER, 
BvarMetricUnit::ROWSETS, "",
+                           "", 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) {}
+
+std::string DorisBvarMetrics::to_prometheus() const {
+    std::stringstream ss;
+    for (auto& entities : entities_map_) {

Review Comment:
   Is it a problem that iterates without locking?
   consider the following situation: the BE is starting up and initializing 
`entities_map_` and Prometheus is scraping at the same time.



##########
be/src/util/system_bvar_metrics.cpp:
##########
@@ -0,0 +1,1129 @@
+// 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", Labels({{"device", cpu_name}, 
{"mode", "user"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_nice, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "nice"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_system, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "system"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_idle, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "idle"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_iowait, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "iowait"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_irq, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "irq"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_soft_irq, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT,
+                                  "", "cpu", Labels({{"device", cpu_name}, 
{"mode", "soft_irq"}}),
+                                  false)
+        DECLARE_INT64_BVAR_METRIC(cpu_steal, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "steal"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_guest, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT, "",
+                                  "cpu", Labels({{"device", cpu_name}, 
{"mode", "guest"}}), false)
+        DECLARE_INT64_BVAR_METRIC(cpu_guest_nice, BvarMetricType::COUNTER, 
BvarMetricUnit::PERCENT,
+                                  "", "cpu", 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,
+                               "", "", Labels(), false)
+        INIT_INT64_BVAR_METRIC(memory_pgpgin, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "", "",
+                               Labels(), false)
+        INIT_INT64_BVAR_METRIC(memory_pgpgout, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "",
+                               "", Labels(), false)
+        INIT_INT64_BVAR_METRIC(memory_pswpin, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "", "",
+                               Labels(), false)
+        INIT_INT64_BVAR_METRIC(memory_pswpout, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "",
+                               "", 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, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_tcmalloc_total_thread_cache_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_tcmalloc_central_cache_free_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_tcmalloc_transfer_cache_free_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_tcmalloc_thread_cache_free_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_free_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_tcmalloc_pageheap_unmapped_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", 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, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_active_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_metadata_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_resident_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_mapped_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_retained_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_tcache_bytes, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_pactive_num, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_pdirty_num, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_pmuzzy_num, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_dirty_purged_num, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", Labels(), false);
+        INIT_INT64_BVAR_METRIC(memory_jemalloc_muzzy_purged_num, 
BvarMetricType::GAUGE,
+                               BvarMetricUnit::BYTES, "", "", 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, "", "",
+                               Labels({{"device", disk_device}}), false)
+        INIT_INT64_BVAR_METRIC(disk_bytes_read, BvarMetricType::COUNTER, 
BvarMetricUnit::BYTES, "",
+                               "", Labels({{"device", disk_device}}), false)
+        INIT_INT64_BVAR_METRIC(disk_read_time_ms, BvarMetricType::COUNTER,
+                               BvarMetricUnit::MILLISECONDS, "", "",
+                               Labels({{"device", disk_device}}), false)
+        INIT_INT64_BVAR_METRIC(disk_writes_completed, BvarMetricType::COUNTER,
+                               BvarMetricUnit::OPERATIONS, "", "",
+                               Labels({{"device", disk_device}}), false)
+        INIT_INT64_BVAR_METRIC(disk_bytes_written, BvarMetricType::COUNTER, 
BvarMetricUnit::BYTES,
+                               "", "", Labels({{"device", disk_device}}), 
false);
+        INIT_INT64_BVAR_METRIC(disk_write_time_ms, BvarMetricType::COUNTER,
+                               BvarMetricUnit::MILLISECONDS, "", "",
+                               Labels({{"device", disk_device}}), false)
+        INIT_INT64_BVAR_METRIC(disk_io_time_ms, BvarMetricType::COUNTER,
+                               BvarMetricUnit::MILLISECONDS, "", "",
+                               Labels({{"device", disk_device}}), false)
+        INIT_INT64_BVAR_METRIC(disk_io_time_weigthed, BvarMetricType::COUNTER,
+                               BvarMetricUnit::MILLISECONDS, "", "",
+                               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, "", "", 
Labels({{"device", interface}}),
+                               false);
+        INIT_INT64_BVAR_METRIC(network_receive_packets, 
BvarMetricType::COUNTER,
+                               BvarMetricUnit::PACKETS, "", "", 
Labels({{"device", interface}}),
+                               false);
+        INIT_INT64_BVAR_METRIC(network_send_bytes, BvarMetricType::COUNTER, 
BvarMetricUnit::BYTES,
+                               "", "", Labels({{"device", interface}}), false);
+        INIT_INT64_BVAR_METRIC(network_send_packets, BvarMetricType::COUNTER,
+                               BvarMetricUnit::PACKETS, "", "", 
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, "", "",
+                               Labels(), false);
+        INIT_INT64_BVAR_METRIC(fd_num_used, BvarMetricType::GAUGE, 
BvarMetricUnit::NOUNIT, "", "",
+                               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", "", Labels(),
+                               false)
+        INIT_INT64_BVAR_METRIC(snmp_tcp_retrans_segs, BvarMetricType::COUNTER,
+                               BvarMetricUnit::NOUNIT, "All TCP packets 
retransmitted", "",
+                               Labels(), false)
+        INIT_INT64_BVAR_METRIC(snmp_tcp_in_segs, BvarMetricType::COUNTER, 
BvarMetricUnit::NOUNIT,
+                               "All received TCP packets", "", Labels(), false)
+        INIT_INT64_BVAR_METRIC(snmp_tcp_out_segs, BvarMetricType::COUNTER, 
BvarMetricUnit::NOUNIT,
+                               "All send TCP packets with RST mark", "", 
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",
+                                Labels({{"mode", "1_minutes"}}), false);
+        INIT_DOUBLE_BVAR_METRIC(load_average_5_minutes, BvarMetricType::GAUGE,
+                                BvarMetricUnit::NOUNIT, "", "load_average",
+                                Labels({{"mode", "5_minutes"}}), false);
+        INIT_DOUBLE_BVAR_METRIC(load_average_15_minutes, BvarMetricType::GAUGE,
+                                BvarMetricUnit::NOUNIT, "", "load_average",
+                                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", Labels({{"mode", "interrupt"}}), false);
+        INIT_INT64_BVAR_METRIC(proc_ctxt_switch, BvarMetricType::COUNTER, 
BvarMetricUnit::NOUNIT,
+                               "", "proc", Labels({{"mode", "ctxt_switch"}}), 
false);
+        INIT_INT64_BVAR_METRIC(proc_procs_running, BvarMetricType::COUNTER, 
BvarMetricUnit::NOUNIT,
+                               "", "proc", Labels({{"mode", 
"procs_running"}}), false);
+        INIT_INT64_BVAR_METRIC(proc_procs_blocked, BvarMetricType::COUNTER, 
BvarMetricUnit::NOUNIT,
+                               "", "proc", 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;
+};
+
+std::string SystemBvarMetrics::to_prometheus(const std::string& registry_name) 
const {
+    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,
+                           "", "", Labels(), true)
+    INIT_INT64_BVAR_METRIC(max_network_send_bytes_rate, BvarMetricType::GAUGE,
+                           BvarMetricUnit::BYTES, "", "", Labels(), true)
+    INIT_INT64_BVAR_METRIC(max_network_receive_bytes_rate, 
BvarMetricType::GAUGE,
+                           BvarMetricUnit::BYTES, "", "", 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() {
+#ifdef BE_TEST
+    // to mock proc
+    FILE* fp = fopen(k_ut_net_dev_path, "r");
+#else
+    FILE* fp = fopen("/proc/net/dev", "r");
+#endif
+    if (fp == nullptr) {
+        char buf[64];
+        LOG(WARNING) << "open /proc/net/dev failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        return;
+    }
+
+    // Ignore header
+    if (getline(&line_ptr_, &line_buf_size_, fp) < 0 ||
+        getline(&line_ptr_, &line_buf_size_, fp) < 0) {
+        char buf[64];
+        LOG(WARNING) << "read /proc/net/dev first two line failed, errno=" << 
errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        fclose(fp);
+        return;
+    }
+    if (proc_net_dev_version_ == 0) {
+        if (strstr(line_ptr_, "compressed") != nullptr) {
+            proc_net_dev_version_ = 3;
+        } else if (strstr(line_ptr_, "bytes") != nullptr) {
+            proc_net_dev_version_ = 2;
+        } else {
+            proc_net_dev_version_ = 1;
+        }
+    }
+
+    while (getline(&line_ptr_, &line_buf_size_, fp) > 0) {
+        char* ptr = strrchr(line_ptr_, ':');
+        if (ptr == nullptr) {
+            continue;
+        }
+        char* start = line_ptr_;
+        while (isspace(*start)) {
+            start++;
+        }
+        std::string interface(start, ptr - start);
+        auto it = network_metrics_.find(interface);
+        if (it == network_metrics_.end()) {
+            continue;
+        }
+        ptr++;
+        int64_t receive_bytes = 0;
+        int64_t receive_packets = 0;
+        int64_t send_bytes = 0;
+        int64_t send_packets = 0;
+        switch (proc_net_dev_version_) {
+        case 3:
+            // receive: bytes packets errs drop fifo frame compressed multicast
+            // send:    bytes packets errs drop fifo colls carrier compressed
+            sscanf(ptr,
+                   " %" PRId64 " %" PRId64
+                   " %*d %*d %*d %*d %*d %*d"
+                   " %" PRId64 " %" PRId64 " %*d %*d %*d %*d %*d %*d",
+                   &receive_bytes, &receive_packets, &send_bytes, 
&send_packets);
+            break;
+        case 2:
+            // receive: bytes packets errs drop fifo frame
+            // send:    bytes packets errs drop fifo colls carrier
+            sscanf(ptr,
+                   " %" PRId64 " %" PRId64
+                   " %*d %*d %*d %*d"
+                   " %" PRId64 " %" PRId64 " %*d %*d %*d %*d %*d",
+                   &receive_bytes, &receive_packets, &send_bytes, 
&send_packets);
+            break;
+        case 1:
+            // receive: packets errs drop fifo frame
+            // send: packets errs drop fifo colls carrier
+            sscanf(ptr,
+                   " %" PRId64
+                   " %*d %*d %*d %*d"
+                   " %" PRId64 " %*d %*d %*d %*d %*d",
+                   &receive_packets, &send_packets);
+            break;
+        default:
+            break;
+        }
+        it->second->network_receive_bytes->set_value(receive_bytes);
+        it->second->network_receive_packets->set_value(receive_packets);
+        it->second->network_send_bytes->set_value(send_bytes);
+        it->second->network_send_packets->set_value(send_packets);
+    }
+    if (ferror(fp) != 0) {
+        char buf[64];
+        LOG(WARNING) << "getline failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+    }
+    fclose(fp);
+}
+
+void SystemBvarMetrics::install_fd_metrics() {
+    auto fd_entity = std::make_shared<BvarMetricEntity>("fd_num", 
BvarMetricType::GAUGE);
+    fd_metrics_ = std::make_shared<FileDescriptorBvarMetrics>(fd_entity);
+    entities_map_["fd_num"].push_back(fd_entity);
+}
+
+void SystemBvarMetrics::update_fd_metrics() {
+#ifdef BE_TEST
+    FILE* fp = fopen(k_ut_fd_path, "r");
+#else
+    FILE* fp = fopen("/proc/sys/fs/file-nr", "r");
+#endif
+    if (fp == nullptr) {
+        char buf[64];
+        LOG(WARNING) << "open /proc/sys/fs/file-nr failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        return;
+    }
+
+    // /proc/sys/fs/file-nr: 
https://www.kernel.org/doc/Documentation/sysctl/fs.txt
+    // 1 - the number of allocated file handles
+    // 2 - the number of allocated but unused file handles
+    // 3 - the maximum number of file handles
+
+    int64_t values[3];
+    if (getline(&line_ptr_, &line_buf_size_, fp) > 0) {
+        memset(values, 0, sizeof(values));
+        int num = sscanf(line_ptr_, "%" PRId64 " %" PRId64 " %" PRId64, 
&values[0], &values[1],
+                         &values[2]);
+        if (num == 3) {
+            fd_metrics_->fd_num_limit->set_value(values[2]);
+            fd_metrics_->fd_num_used->set_value(values[0] - values[1]);
+        }
+    }
+
+    if (ferror(fp) != 0) {
+        char buf[64];
+        LOG(WARNING) << "getline failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+    }
+    fclose(fp);
+}
+
+void SystemBvarMetrics::install_snmp_metrics() {
+    auto snmp_entity = std::make_shared<BvarMetricEntity>("snmp", 
BvarMetricType::COUNTER);
+    snmp_metrics_ = std::make_shared<SnmpBvarMetrics>(snmp_entity);
+    entities_map_["snmp"].push_back(snmp_entity);
+}
+
+void SystemBvarMetrics::update_snmp_metrics() {
+#ifdef BE_TEST
+    // to mock proc
+    FILE* fp = fopen(k_ut_net_snmp_path, "r");
+#else
+    FILE* fp = fopen("/proc/net/snmp", "r");
+#endif
+    if (fp == nullptr) {
+        char buf[64];
+        LOG(WARNING) << "open /proc/net/snmp failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        return;
+    }
+
+    // We only care about Tcp lines, so skip other lines in front of Tcp line
+    int res = 0;
+    while ((res = getline(&line_ptr_, &line_buf_size_, fp)) > 0) {
+        if (strstr(line_ptr_, "Tcp") != nullptr) {
+            break;
+        }
+    }
+    if (res <= 0) {
+        char buf[64];
+        LOG(WARNING) << "failed to skip lines of /proc/net/snmp, errno=" << 
errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        fclose(fp);
+        return;
+    }
+
+    // parse the Tcp header
+    // Tcp: RtoAlgorithm RtoMin RtoMax MaxConn ActiveOpens PassiveOpens 
AttemptFails EstabResets CurrEstab InSegs OutSegs RetransSegs InErrs OutRsts 
InCsumErrors
+    std::vector<std::string> headers = strings::Split(line_ptr_, " ");
+    std::unordered_map<std::string, int32_t> header_map;
+    int32_t pos = 0;
+    for (auto& h : headers) {
+        header_map.emplace(h, pos++);
+    }
+
+    // read the metrics of TCP
+    if (getline(&line_ptr_, &line_buf_size_, fp) < 0) {
+        char buf[64];
+        LOG(WARNING) << "failed to skip Tcp header line of /proc/net/snmp, 
errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        fclose(fp);
+        return;
+    }
+
+    // metric line looks like:
+    // Tcp: 1 200 120000 -1 47849374 38601877 3353843 2320314 276 1033354613 
1166025166 825439 12694 23238924 0
+    std::vector<std::string> metrics = strings::Split(line_ptr_, " ");
+    if (metrics.size() != headers.size()) {
+        LOG(WARNING) << "invalid tcp metrics line: " << line_ptr_;
+        fclose(fp);
+        return;
+    }
+    int64_t retrans_segs = atoi64(metrics[header_map["RetransSegs"]]);
+    int64_t in_errs = atoi64(metrics[header_map["InErrs"]]);
+    int64_t in_segs = atoi64(metrics[header_map["InSegs"]]);
+    int64_t out_segs = atoi64(metrics[header_map["OutSegs"]]);
+    snmp_metrics_->snmp_tcp_retrans_segs->set_value(retrans_segs);
+    snmp_metrics_->snmp_tcp_in_errs->set_value(in_errs);
+    snmp_metrics_->snmp_tcp_in_segs->set_value(in_segs);
+    snmp_metrics_->snmp_tcp_out_segs->set_value(out_segs);
+
+    if (ferror(fp) != 0) {
+        char buf[64];
+        LOG(WARNING) << "getline failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+    }
+    fclose(fp);
+}
+
+void SystemBvarMetrics::install_load_avg_metrics() {
+    auto load_average_entity =
+            std::make_shared<BvarMetricEntity>("load_average", 
BvarMetricType::COUNTER);
+    load_average_metrics_ = 
std::make_shared<LoadAverageBvarMetrics>(load_average_entity);
+    entities_map_["load_average"].push_back(load_average_entity);
+}
+
+void SystemBvarMetrics::update_load_avg_metrics() {
+#ifdef BE_TEST
+    FILE* fp = fopen(k_ut_load_avg_path, "r");
+#else
+    FILE* fp = fopen("/proc/loadavg", "r");
+#endif
+    if (fp == nullptr) {
+        char buf[64];
+        LOG(WARNING) << "open /proc/loadavg failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        return;
+    }
+
+    double values[3];
+    if (getline(&line_ptr_, &line_buf_size_, fp) > 0) {
+        memset(values, 0, sizeof(values));
+        int num = sscanf(line_ptr_, "%lf %lf %lf", &values[0], &values[1], 
&values[2]);
+        if (num == 3) {
+            
load_average_metrics_->load_average_1_minutes->set_value(values[0]);
+            
load_average_metrics_->load_average_5_minutes->set_value(values[1]);
+            
load_average_metrics_->load_average_15_minutes->set_value(values[2]);
+        }
+    }
+
+    if (ferror(fp) != 0) {
+        char buf[64];
+        LOG(WARNING) << "getline failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+    }
+    fclose(fp);
+}
+
+void SystemBvarMetrics::install_proc_metrics() {
+    auto proc_entity = std::make_shared<BvarMetricEntity>("proc", 
BvarMetricType::COUNTER);
+    proc_metrics_ = std::make_shared<ProcBvarMetrics>(proc_entity);
+    entities_map_["load_average"].push_back(proc_entity);
+}
+
+void SystemBvarMetrics::update_proc_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;
+    }
+
+    uint64_t inter = 0, ctxt = 0, procs_r = 0, procs_b = 0;
+    while (getline(&line_ptr_, &line_buf_size_, fp) > 0) {
+        char* start_pos = nullptr;
+        start_pos = strstr(line_ptr_, "intr ");
+        if (start_pos) {
+            sscanf(start_pos, "intr %" PRIu64, &inter);
+            proc_metrics_->proc_interrupt->set_value(inter);
+        }
+
+        start_pos = strstr(line_ptr_, "ctxt ");
+        if (start_pos) {
+            sscanf(start_pos, "ctxt %" PRIu64, &ctxt);
+            proc_metrics_->proc_ctxt_switch->set_value(ctxt);
+        }
+
+        start_pos = strstr(line_ptr_, "procs_running ");
+        if (start_pos) {
+            sscanf(start_pos, "procs_running %" PRIu64, &procs_r);
+            proc_metrics_->proc_procs_running->set_value(procs_r);
+        }
+
+        start_pos = strstr(line_ptr_, "procs_blocked ");
+        if (start_pos) {
+            sscanf(start_pos, "procs_blocked %" PRIu64, &procs_b);
+            proc_metrics_->proc_procs_blocked->set_value(procs_b);
+        }
+    }
+
+    if (ferror(fp) != 0) {
+        char buf[64];
+        LOG(WARNING) << "getline failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+    }
+
+    fclose(fp);
+}
+
+void SystemBvarMetrics::get_metrics_from_proc_vmstat() {
+#ifdef BE_TEST
+    FILE* fp = fopen(k_ut_vmstat_path, "r");
+#else
+    FILE* fp = fopen("/proc/vmstat", "r");
+#endif
+    if (fp == nullptr) {
+        char buf[64];
+        LOG(WARNING) << "open /proc/vmstat failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+        return;
+    }
+
+    while (getline(&line_ptr_, &line_buf_size_, fp) > 0) {
+        uint64_t value;
+        char name[64];
+        int num = sscanf(line_ptr_, "%s %" PRIu64, name, &value);
+        if (num < 2) {
+            continue;
+        }
+
+        if (strcmp(name, "pgpgin") == 0) {
+            memory_metrics_->memory_pgpgin->set_value(value);
+        } else if (strcmp(name, "pgpgout") == 0) {
+            memory_metrics_->memory_pgpgout->set_value(value);
+        } else if (strcmp(name, "pswpin") == 0) {
+            memory_metrics_->memory_pswpin->set_value(value);
+        } else if (strcmp(name, "pswpout") == 0) {
+            memory_metrics_->memory_pswpout->set_value(value);
+        }
+    }
+
+    if (ferror(fp) != 0) {
+        char buf[64];
+        LOG(WARNING) << "getline failed, errno=" << errno
+                     << ", message=" << strerror_r(errno, buf, 64);
+    }
+
+    fclose(fp);
+}
+
+void SystemBvarMetrics::get_cpu_name() {
+#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];
+        char* start_pos = nullptr;
+        start_pos = strstr(line_ptr_, "cpu");
+        if (start_pos) {
+            sscanf(line_ptr_, "%15s", cpu);
+            std::string cpu_name(cpu);

Review Comment:
   better add a size limit to the ctor.`cpu_name(cpu, 15)` because  `char 
cpu[16]` is not initialized, or just use a std::string(16, '\0') as buffer.



-- 
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


Reply via email to