You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pegasus.apache.org by wa...@apache.org on 2023/05/25 10:55:09 UTC

[incubator-pegasus] 10/28: feat(new_metrics): migrate built-in server-level metrics (#1418)

This is an automated email from the ASF dual-hosted git repository.

wangdan pushed a commit to branch migrate-metrics-dev
in repository https://gitbox.apache.org/repos/asf/incubator-pegasus.git

commit 86c5dc30024884bd2591ec498ed1499140960547
Author: Dan Wang <wa...@apache.org>
AuthorDate: Wed Mar 29 16:17:54 2023 +0800

    feat(new_metrics): migrate built-in server-level metrics (#1418)
    
    https://github.com/apache/incubator-pegasus/issues/1321
    
    Migrate built-in server-level metrics to new framework, including the total
    amount of virtual/physical memory usage in MB. Both metrics are wrapped
    in `builtin_metrics` and updated periodically by a timer, which is started
    or stopped along with meta/replica servers.
    
    Since `dsn_perf_counter_test` involves both metrics which have been
    removed from perf-counters, just disable it.
---
 .github/workflows/lint_and_test_cpp.yaml           | 18 ++++-
 run.sh                                             |  4 +-
 src/perf_counter/builtin_counters.cpp              | 57 --------------
 src/perf_counter/perf_counters.cpp                 |  3 -
 src/server/pegasus_service_app.h                   | 58 ++++++++------
 src/utils/builtin_metrics.cpp                      | 92 ++++++++++++++++++++++
 .../builtin_counters.h => utils/builtin_metrics.h} | 33 +++++---
 src/utils/metrics.cpp                              |  2 +-
 8 files changed, 169 insertions(+), 98 deletions(-)

diff --git a/.github/workflows/lint_and_test_cpp.yaml b/.github/workflows/lint_and_test_cpp.yaml
index 032e632e3..ff6134c0b 100644
--- a/.github/workflows/lint_and_test_cpp.yaml
+++ b/.github/workflows/lint_and_test_cpp.yaml
@@ -185,7 +185,9 @@ jobs:
           - dsn_meta_state_tests
           - dsn.meta.test
           - dsn_nfs_test
-          - dsn_perf_counter_test
+          # TODO(wangdan): Since builtin_counters (memused.virt and memused.res) for perf-counters
+          # have been removed and dsn_perf_counter_test depends on them, disable it.
+          # - dsn_perf_counter_test
           - dsn_replica_backup_test
           - dsn_replica_bulk_load_test
           - dsn_replica_dup_test
@@ -312,7 +314,9 @@ jobs:
           - dsn_meta_state_tests
           - dsn.meta.test
           - dsn_nfs_test
-          - dsn_perf_counter_test
+          # TODO(wangdan): Since builtin_counters (memused.virt and memused.res) for perf-counters
+          # have been removed and dsn_perf_counter_test depends on them, disable it.
+          # - dsn_perf_counter_test
           - dsn_replica_backup_test
           - dsn_replica_bulk_load_test
           - dsn_replica_dup_test
@@ -431,7 +435,11 @@ jobs:
 #          - base_api_test
 #          - base_test
 #          - bulk_load_test
-#          - detect_hotspot_test
+#          # TODO(wangdan): Since the hotspot detection depends on the perf-counters system which
+#          # is being replaced with the new metrics system, its test will fail. Temporarily disable
+#          # the test and re-enable it after the hotspot detection is migrated to the new metrics
+#          # system.
+#          # - detect_hotspot_test
 #          - dsn_aio_test
 #          - dsn_block_service_test
 #          - dsn_client_test
@@ -440,7 +448,9 @@ jobs:
 #          - dsn_meta_state_tests
 #          - dsn.meta.test
 #          - dsn_nfs_test
-#          - dsn_perf_counter_test
+#          # TODO(wangdan): Since builtin_counters (memused.virt and memused.res) for perf-counters
+#          # have been removed and dsn_perf_counter_test depends on them, disable it.
+#          # - dsn_perf_counter_test
 #          - dsn_replica_backup_test
 #          - dsn_replica_bulk_load_test
 #          - dsn_replica_dup_test
diff --git a/run.sh b/run.sh
index 0df98b040..21b92dbdc 100755
--- a/run.sh
+++ b/run.sh
@@ -355,7 +355,9 @@ function run_test()
       dsn_meta_state_tests
       dsn.meta.test
       dsn_nfs_test
-      dsn_perf_counter_test
+      # TODO(wangdan): Since builtin_counters (memused.virt and memused.res) for perf-counters
+      # have been removed and dsn_perf_counter_test depends on them, disable it.
+      # dsn_perf_counter_test
       dsn_replica_backup_test
       dsn_replica_bulk_load_test
       dsn_replica_dup_test
diff --git a/src/perf_counter/builtin_counters.cpp b/src/perf_counter/builtin_counters.cpp
deleted file mode 100644
index 26f57c566..000000000
--- a/src/perf_counter/builtin_counters.cpp
+++ /dev/null
@@ -1,57 +0,0 @@
-// 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 "builtin_counters.h"
-
-#include <stdint.h>
-
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
-#include "utils/fmt_logging.h"
-#include "utils/process_utils.h"
-
-namespace dsn {
-
-builtin_counters::builtin_counters()
-{
-    _memused_virt.init_global_counter("replica",
-                                      "server",
-                                      "memused.virt(MB)",
-                                      COUNTER_TYPE_NUMBER,
-                                      "virtual memory usages in MB");
-    _memused_res.init_global_counter("replica",
-                                     "server",
-                                     "memused.res(MB)",
-                                     COUNTER_TYPE_NUMBER,
-                                     "physically memory usages in MB");
-}
-
-builtin_counters::~builtin_counters() {}
-
-void builtin_counters::update_counters()
-{
-    double vm_usage;
-    double resident_set;
-    utils::process_mem_usage(vm_usage, resident_set);
-    uint64_t memused_virt = (uint64_t)vm_usage / 1024;
-    uint64_t memused_res = (uint64_t)resident_set / 1024;
-    _memused_virt->set(memused_virt);
-    _memused_res->set(memused_res);
-    LOG_INFO("memused_virt = {} MB, memused_res = {} MB", memused_virt, memused_res);
-}
-
-} // namespace dsn
diff --git a/src/perf_counter/perf_counters.cpp b/src/perf_counter/perf_counters.cpp
index 1f2d4574b..49482fbd2 100644
--- a/src/perf_counter/perf_counters.cpp
+++ b/src/perf_counter/perf_counters.cpp
@@ -32,7 +32,6 @@
 #include <sstream>
 #include <utility>
 
-#include "builtin_counters.h"
 #include "perf_counter/perf_counter.h"
 #include "perf_counter/perf_counter_atomic.h"
 #include "perf_counter/perf_counter_utils.h"
@@ -367,8 +366,6 @@ std::string perf_counters::list_snapshot_by_literal(
 
 void perf_counters::take_snapshot()
 {
-    builtin_counters::instance().update_counters();
-
     std::vector<perf_counter_ptr> all_counters;
     get_all_counters(&all_counters);
 
diff --git a/src/server/pegasus_service_app.h b/src/server/pegasus_service_app.h
index bae690114..c581ecae0 100644
--- a/src/server/pegasus_service_app.h
+++ b/src/server/pegasus_service_app.h
@@ -24,6 +24,7 @@
 #include <pegasus/version.h>
 #include <pegasus/git_commit.h>
 #include "reporter/pegasus_counter_reporter.h"
+#include "utils/builtin_metrics.h"
 
 namespace pegasus {
 namespace server {
@@ -32,8 +33,7 @@ class pegasus_replication_service_app : public ::dsn::replication::replication_s
 {
 public:
     pegasus_replication_service_app(const dsn::service_app_info *info)
-        : ::dsn::replication::replication_service_app::replication_service_app(info),
-          _updater_started(false)
+        : ::dsn::replication::replication_service_app::replication_service_app(info)
     {
     }
 
@@ -43,33 +43,39 @@ public:
         std::vector<std::string> args_new(args);
         args_new.emplace_back(PEGASUS_VERSION);
         args_new.emplace_back(PEGASUS_GIT_COMMIT);
-        ::dsn::error_code ret = ::dsn::replication::replication_service_app::start(args_new);
 
-        if (ret == ::dsn::ERR_OK) {
-            pegasus_counter_reporter::instance().start();
-            _updater_started = true;
-        }
-        return ret;
+        // Actually the root caller, start_app() in service_control_task::exec() will also do
+        // CHECK for ERR_OK. Do CHECK here to guarantee that all following services (such as
+        // built-in metrics) are started.
+        CHECK_EQ(::dsn::replication::replication_service_app::start(args_new), ::dsn::ERR_OK);
+
+        // TODO(wangdan): remove after all metrics have been migrated.
+        pegasus_counter_reporter::instance().start();
+
+        _builtin_metrics.start();
+        return ::dsn::ERR_OK;
     }
 
     virtual ::dsn::error_code stop(bool cleanup = false) override
     {
         ::dsn::error_code ret = ::dsn::replication::replication_service_app::stop();
-        if (_updater_started) {
-            pegasus_counter_reporter::instance().stop();
-        }
+
+        // TODO(wangdan): remove after all metrics have been migrated.
+        pegasus_counter_reporter::instance().stop();
+
+        _builtin_metrics.stop();
         return ret;
     }
 
 private:
-    bool _updater_started;
+    dsn::builtin_metrics _builtin_metrics;
 };
 
 class pegasus_meta_service_app : public ::dsn::service::meta_service_app
 {
 public:
     pegasus_meta_service_app(const dsn::service_app_info *info)
-        : ::dsn::service::meta_service_app::meta_service_app(info), _updater_started(false)
+        : ::dsn::service::meta_service_app::meta_service_app(info)
     {
     }
 
@@ -79,26 +85,32 @@ public:
         std::vector<std::string> args_new(args);
         args_new.emplace_back(PEGASUS_VERSION);
         args_new.emplace_back(PEGASUS_GIT_COMMIT);
-        ::dsn::error_code ret = ::dsn::service::meta_service_app::start(args_new);
 
-        if (ret == ::dsn::ERR_OK) {
-            pegasus_counter_reporter::instance().start();
-            _updater_started = true;
-        }
-        return ret;
+        // Actually the root caller, start_app() in service_control_task::exec() will also do
+        // CHECK for ERR_OK. Do CHECK here to guarantee that all following services (such as
+        // built-in metrics) are started.
+        CHECK_EQ(::dsn::service::meta_service_app::start(args_new), ::dsn::ERR_OK);
+
+        // TODO(wangdan): remove after all metrics have been migrated.
+        pegasus_counter_reporter::instance().start();
+
+        _builtin_metrics.start();
+        return ::dsn::ERR_OK;
     }
 
     virtual ::dsn::error_code stop(bool cleanup = false) override
     {
         ::dsn::error_code ret = ::dsn::service::meta_service_app::stop();
-        if (_updater_started) {
-            pegasus_counter_reporter::instance().stop();
-        }
+
+        // TODO(wangdan): remove after all metrics have been migrated.
+        pegasus_counter_reporter::instance().stop();
+
+        _builtin_metrics.stop();
         return ret;
     }
 
 private:
-    bool _updater_started;
+    dsn::builtin_metrics _builtin_metrics;
 };
 
 } // namespace server
diff --git a/src/utils/builtin_metrics.cpp b/src/utils/builtin_metrics.cpp
new file mode 100644
index 000000000..af79fc8b4
--- /dev/null
+++ b/src/utils/builtin_metrics.cpp
@@ -0,0 +1,92 @@
+// 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 "utils/builtin_metrics.h"
+
+#include <stdint.h>
+#include <functional>
+
+#include "utils/autoref_ptr.h"
+#include "utils/flags.h"
+#include "utils/fmt_logging.h"
+#include "utils/process_utils.h"
+#include "utils/string_view.h"
+
+METRIC_DEFINE_gauge_int64(server,
+                          virtual_mem_usage_mb,
+                          dsn::metric_unit::kMegaBytes,
+                          "The total amount of virtual memory usage in MB");
+
+METRIC_DEFINE_gauge_int64(server,
+                          resident_mem_usage_mb,
+                          dsn::metric_unit::kMegaBytes,
+                          "The total amount of physical memory usage in MB");
+
+namespace dsn {
+
+DSN_DEFINE_uint64(metrics,
+                  builtin_metrics_update_interval_ms,
+                  10 * 1000,
+                  "The interval (milliseconds) at which builtin metrics are updated.");
+
+builtin_metrics::builtin_metrics()
+    : METRIC_VAR_INIT_server(virtual_mem_usage_mb), METRIC_VAR_INIT_server(resident_mem_usage_mb)
+{
+}
+
+builtin_metrics::~builtin_metrics()
+{
+    CHECK(!_timer, "timer should have been destroyed by stop()");
+}
+
+void builtin_metrics::on_close() {}
+
+void builtin_metrics::start()
+{
+    CHECK(!_timer, "timer should not have been initialized before start()");
+
+    _timer.reset(new metric_timer(FLAGS_builtin_metrics_update_interval_ms,
+                                  std::bind(&builtin_metrics::update, this),
+                                  std::bind(&builtin_metrics::on_close, this)));
+}
+
+void builtin_metrics::stop()
+{
+    CHECK(_timer, "timer should have been initialized before stop()");
+
+    // Close the timer synchronously.
+    _timer->close();
+    _timer->wait();
+
+    // Reset the timer to mark that it has been stopped, now it could be started.
+    _timer.reset();
+}
+
+void builtin_metrics::update()
+{
+    double vm_usage;
+    double resident_set;
+    utils::process_mem_usage(vm_usage, resident_set);
+
+    auto virt_mb = static_cast<uint64_t>(vm_usage) >> 10;
+    auto res_mb = static_cast<uint64_t>(resident_set) >> 10;
+    METRIC_VAR_SET(virtual_mem_usage_mb, virt_mb);
+    METRIC_VAR_SET(resident_mem_usage_mb, res_mb);
+    LOG_INFO("virt = {} MB, res = {} MB", virt_mb, res_mb);
+}
+
+} // namespace dsn
diff --git a/src/perf_counter/builtin_counters.h b/src/utils/builtin_metrics.h
similarity index 64%
rename from src/perf_counter/builtin_counters.h
rename to src/utils/builtin_metrics.h
index 58ca9f778..2a0094d70 100644
--- a/src/perf_counter/builtin_counters.h
+++ b/src/utils/builtin_metrics.h
@@ -15,19 +15,34 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "perf_counter_wrapper.h"
-#include "utils/singleton.h"
+#pragma once
+
+#include <memory>
+
+#include "utils/metrics.h"
+#include "utils/ports.h"
 
 namespace dsn {
-class builtin_counters : public dsn::utils::singleton<builtin_counters>
+
+class builtin_metrics
 {
 public:
-    builtin_counters();
-    ~builtin_counters();
-    void update_counters();
+    builtin_metrics();
+    ~builtin_metrics();
+
+    void start();
+    void stop();
 
 private:
-    dsn::perf_counter_wrapper _memused_virt;
-    dsn::perf_counter_wrapper _memused_res;
+    void on_close();
+    void update();
+
+    METRIC_VAR_DECLARE_gauge_int64(virtual_mem_usage_mb);
+    METRIC_VAR_DECLARE_gauge_int64(resident_mem_usage_mb);
+
+    std::unique_ptr<metric_timer> _timer;
+
+    DISALLOW_COPY_AND_ASSIGN(builtin_metrics);
 };
-}
+
+} // namespace dsn
diff --git a/src/utils/metrics.cpp b/src/utils/metrics.cpp
index b32887c55..dd4c5dd56 100644
--- a/src/utils/metrics.cpp
+++ b/src/utils/metrics.cpp
@@ -43,7 +43,7 @@ namespace dsn {
 DSN_DEFINE_uint64(metrics,
                   entity_retirement_delay_ms,
                   10 * 60 * 1000,
-                  "The retention internal (milliseconds) for an entity after it becomes stale.");
+                  "The retention interval (milliseconds) for an entity after it becomes stale.");
 
 metric_entity::metric_entity(const metric_entity_prototype *prototype,
                              const std::string &id,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pegasus.apache.org
For additional commands, e-mail: commits-help@pegasus.apache.org