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:23 UTC

[incubator-pegasus] 24/28: feat(new_metrics): migrate metrics for replica_stub (part 5) (#1469)

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 5810ccd7702d167014ea6300bc84bf963da8101c
Author: Dan Wang <wa...@apache.org>
AuthorDate: Thu May 11 11:32:43 2023 +0800

    feat(new_metrics): migrate metrics for replica_stub (part 5) (#1469)
    
    https://github.com/apache/incubator-pegasus/issues/1454
    
    This is the 5th part of migrating metrics of replica_stub to new framework,
    most of which are cold-backup-related.
    
    During this migration, there are 10 metrics which are changed from server-level
    to replica-level, including the number of current running backups, the max backup
    duration among backups, the max size of uploaded files among backups, the number
    of started/failed/successful/cancelled backups, the number of failed/successful file
    uploads for backups, the total size of uploaded files for backups.
    
    The metric tracking the number of paused cold backup has been removed, since
    it's not in use. `tcmalloc_released_bytes` is changed from gauge to counter, since
    the counter would be a more accurate reflection of recently released memory by
    tcmalloc.
---
 src/meta/meta_backup_service.cpp              |   8 +-
 src/meta/meta_backup_service.h                |   4 +-
 src/replica/backup/cold_backup_context.cpp    |  28 +++----
 src/replica/backup/replica_backup_manager.cpp |  32 +++++--
 src/replica/backup/replica_backup_manager.h   |   9 +-
 src/replica/replica.cpp                       |  47 ++++++++++-
 src/replica/replica.h                         |  19 +++--
 src/replica/replica_backup.cpp                |   7 +-
 src/replica/replica_stub.cpp                  | 116 ++++++--------------------
 src/replica/replica_stub.h                    |  14 +---
 src/utils/metrics.h                           |   2 +
 11 files changed, 140 insertions(+), 146 deletions(-)

diff --git a/src/meta/meta_backup_service.cpp b/src/meta/meta_backup_service.cpp
index cfdddbb53..832158161 100644
--- a/src/meta/meta_backup_service.cpp
+++ b/src/meta/meta_backup_service.cpp
@@ -55,9 +55,9 @@
 METRIC_DEFINE_entity(backup_policy);
 
 METRIC_DEFINE_gauge_int64(backup_policy,
-                          policy_recent_backup_duration_ms,
+                          backup_recent_duration_ms,
                           dsn::metric_unit::kMilliSeconds,
-                          "The recent backup duration");
+                          "The duration of recent backup");
 
 namespace dsn {
 namespace replication {
@@ -78,7 +78,7 @@ metric_entity_ptr instantiate_backup_policy_metric_entity(const std::string &pol
 
 backup_policy_metrics::backup_policy_metrics(const std::string &policy_name)
     : _backup_policy_metric_entity(instantiate_backup_policy_metric_entity(policy_name)),
-      METRIC_VAR_INIT_backup_policy(policy_recent_backup_duration_ms)
+      METRIC_VAR_INIT_backup_policy(backup_recent_duration_ms)
 {
 }
 
@@ -1039,7 +1039,7 @@ void policy_context::issue_gc_backup_info_task_unlocked()
             last_backup_duration_time_ms = (_cur_backup.end_time_ms - _cur_backup.start_time_ms);
         }
     }
-    METRIC_SET(*_metrics, policy_recent_backup_duration_ms, last_backup_duration_time_ms);
+    METRIC_SET(*_metrics, backup_recent_duration_ms, last_backup_duration_time_ms);
 }
 
 void policy_context::sync_remove_backup_info(const backup_info &info, dsn::task_ptr sync_callback)
diff --git a/src/meta/meta_backup_service.h b/src/meta/meta_backup_service.h
index 021b99552..14afbf7f4 100644
--- a/src/meta/meta_backup_service.h
+++ b/src/meta/meta_backup_service.h
@@ -184,11 +184,11 @@ public:
 
     const metric_entity_ptr &backup_policy_metric_entity() const;
 
-    METRIC_DEFINE_SET(policy_recent_backup_duration_ms, int64_t)
+    METRIC_DEFINE_SET(backup_recent_duration_ms, int64_t)
 
 private:
     const metric_entity_ptr _backup_policy_metric_entity;
-    METRIC_VAR_DECLARE_gauge_int64(policy_recent_backup_duration_ms);
+    METRIC_VAR_DECLARE_gauge_int64(backup_recent_duration_ms);
 
     DISALLOW_COPY_AND_ASSIGN(backup_policy_metrics);
 };
diff --git a/src/replica/backup/cold_backup_context.cpp b/src/replica/backup/cold_backup_context.cpp
index f9063a896..5ff49e862 100644
--- a/src/replica/backup/cold_backup_context.cpp
+++ b/src/replica/backup/cold_backup_context.cpp
@@ -24,15 +24,13 @@
 
 #include "common/backup_common.h"
 #include "common/replication.codes.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica.h"
-#include "replica/replica_stub.h"
 #include "runtime/api_layer1.h"
 #include "runtime/task/async_calls.h"
 #include "utils/blob.h"
 #include "utils/error_code.h"
 #include "utils/filesystem.h"
+#include "utils/metrics.h"
 #include "utils/utils.h"
 
 namespace dsn {
@@ -71,7 +69,7 @@ void cold_backup_context::cancel()
 {
     _status.store(ColdBackupCanceled);
     if (_owner_replica != nullptr) {
-        _owner_replica->get_replica_stub()->_counter_cold_backup_recent_cancel_count->increment();
+        METRIC_INCREMENT(*_owner_replica, backup_cancelled_count);
     }
 }
 
@@ -93,7 +91,7 @@ bool cold_backup_context::fail_check(const char *failure_reason)
         strncpy(_reason, failure_reason, sizeof(_reason) - 1);
         _reason[sizeof(_reason) - 1] = '\0';
         if (_owner_replica != nullptr) {
-            _owner_replica->get_replica_stub()->_counter_cold_backup_recent_fail_count->increment();
+            METRIC_INCREMENT(*_owner_replica, backup_failed_count);
         }
         return true;
     } else {
@@ -107,7 +105,7 @@ bool cold_backup_context::complete_check(bool uploaded)
     if (uploaded) {
         _progress.store(cold_backup_constant::PROGRESS_FINISHED);
         if (_owner_replica != nullptr) {
-            _owner_replica->get_replica_stub()->_counter_cold_backup_recent_succ_count->increment();
+            METRIC_INCREMENT(*_owner_replica, backup_successful_count);
         }
         return _status.compare_exchange_strong(checking, ColdBackupCompleted);
     } else {
@@ -132,7 +130,7 @@ bool cold_backup_context::fail_checkpoint(const char *failure_reason)
         strncpy(_reason, failure_reason, sizeof(_reason) - 1);
         _reason[sizeof(_reason) - 1] = '\0';
         if (_owner_replica != nullptr) {
-            _owner_replica->get_replica_stub()->_counter_cold_backup_recent_fail_count->increment();
+            METRIC_INCREMENT(*_owner_replica, backup_failed_count);
         }
         return true;
     } else {
@@ -158,7 +156,7 @@ bool cold_backup_context::fail_upload(const char *failure_reason)
         strncpy(_reason, failure_reason, sizeof(_reason) - 1);
         _reason[sizeof(_reason) - 1] = '\0';
         if (_owner_replica != nullptr) {
-            _owner_replica->get_replica_stub()->_counter_cold_backup_recent_fail_count->increment();
+            METRIC_INCREMENT(*_owner_replica, backup_failed_count);
         }
         return true;
     } else {
@@ -174,7 +172,7 @@ bool cold_backup_context::complete_upload()
         _status.compare_exchange_strong(paused, ColdBackupCompleted)) {
         _progress.store(cold_backup_constant::PROGRESS_FINISHED);
         if (_owner_replica != nullptr) {
-            _owner_replica->get_replica_stub()->_counter_cold_backup_recent_succ_count->increment();
+            METRIC_INCREMENT(*_owner_replica, backup_successful_count);
         }
         return true;
     } else {
@@ -720,8 +718,7 @@ void cold_backup_context::upload_file(const std::string &local_filename)
                 fail_upload("create file failed");
             }
             if (resp.err != ERR_OK && _owner_replica != nullptr) {
-                _owner_replica->get_replica_stub()
-                    ->_counter_cold_backup_recent_upload_file_fail_count->increment();
+                METRIC_INCREMENT(*_owner_replica, backup_file_upload_failed_count);
             }
             release_ref();
             return;
@@ -783,8 +780,7 @@ void cold_backup_context::on_upload(const dist::block_service::block_file_ptr &f
                 fail_upload("upload checkpoint file to remote failed");
             }
             if (resp.err != ERR_OK && _owner_replica != nullptr) {
-                _owner_replica->get_replica_stub()
-                    ->_counter_cold_backup_recent_upload_file_fail_count->increment();
+                METRIC_INCREMENT(*_owner_replica, backup_file_upload_failed_count);
             }
             release_ref();
             return;
@@ -1003,10 +999,8 @@ void cold_backup_context::on_upload_file_complete(const std::string &local_filen
     _upload_file_size.fetch_add(f_size);
     file_upload_complete(local_filename);
     if (_owner_replica != nullptr) {
-        _owner_replica->get_replica_stub()
-            ->_counter_cold_backup_recent_upload_file_succ_count->increment();
-        _owner_replica->get_replica_stub()->_counter_cold_backup_recent_upload_file_size->add(
-            f_size);
+        METRIC_INCREMENT(*_owner_replica, backup_file_upload_successful_count);
+        METRIC_INCREMENT_BY(*_owner_replica, backup_file_upload_total_bytes, f_size);
     }
     // update progress
     // int a = 10; int b = 3; then  b/a = 0;
diff --git a/src/replica/backup/replica_backup_manager.cpp b/src/replica/backup/replica_backup_manager.cpp
index 0d34afa2d..6ab2af844 100644
--- a/src/replica/backup/replica_backup_manager.cpp
+++ b/src/replica/backup/replica_backup_manager.cpp
@@ -19,7 +19,6 @@
 
 #include <stdint.h>
 #include <algorithm>
-#include <atomic>
 #include <chrono>
 #include <map>
 #include <memory>
@@ -41,9 +40,25 @@
 #include "utils/filesystem.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
 #include "utils/strings.h"
 #include "utils/thread_access_checker.h"
 
+METRIC_DEFINE_gauge_int64(replica,
+                          backup_running_count,
+                          dsn::metric_unit::kBackups,
+                          "The number of current running backups");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          backup_max_duration_ms,
+                          dsn::metric_unit::kMilliSeconds,
+                          "The max backup duration among backups");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          backup_file_upload_max_bytes,
+                          dsn::metric_unit::kBytes,
+                          "The max size of uploaded files among backups");
+
 namespace dsn {
 namespace replication {
 
@@ -83,7 +98,14 @@ static bool get_policy_checkpoint_dirs(const std::string &dir,
     return true;
 }
 
-replica_backup_manager::replica_backup_manager(replica *r) : replica_base(r), _replica(r) {}
+replica_backup_manager::replica_backup_manager(replica *r)
+    : replica_base(r),
+      _replica(r),
+      METRIC_VAR_INIT_replica(backup_running_count),
+      METRIC_VAR_INIT_replica(backup_max_duration_ms),
+      METRIC_VAR_INIT_replica(backup_file_upload_max_bytes)
+{
+}
 
 replica_backup_manager::~replica_backup_manager()
 {
@@ -160,9 +182,9 @@ void replica_backup_manager::collect_backup_info()
         }
     }
 
-    _replica->_cold_backup_running_count.store(cold_backup_running_count);
-    _replica->_cold_backup_max_duration_time_ms.store(cold_backup_max_duration_time_ms);
-    _replica->_cold_backup_max_upload_file_size.store(cold_backup_max_upload_file_size);
+    METRIC_VAR_SET(backup_running_count, cold_backup_running_count);
+    METRIC_VAR_SET(backup_max_duration_ms, cold_backup_max_duration_time_ms);
+    METRIC_VAR_SET(backup_file_upload_max_bytes, cold_backup_max_upload_file_size);
 }
 
 void replica_backup_manager::background_clear_backup_checkpoint(const std::string &policy_name)
diff --git a/src/replica/backup/replica_backup_manager.h b/src/replica/backup/replica_backup_manager.h
index 80c479d16..40d005537 100644
--- a/src/replica/backup/replica_backup_manager.h
+++ b/src/replica/backup/replica_backup_manager.h
@@ -21,6 +21,7 @@
 
 #include "replica/replica_base.h"
 #include "runtime/task/task.h"
+#include "utils/metrics.h"
 
 namespace dsn {
 class gpid;
@@ -40,6 +41,9 @@ public:
     void start_collect_backup_info();
 
 private:
+    friend class replica;
+    friend class replica_backup_manager_test;
+
     void clear_backup_checkpoint(const std::string &policy_name);
     void send_clear_request_to_secondaries(const gpid &pid, const std::string &policy_name);
     void background_clear_backup_checkpoint(const std::string &policy_name);
@@ -48,8 +52,9 @@ private:
     replica *_replica;
     dsn::task_ptr _collect_info_timer;
 
-    friend class replica;
-    friend class replica_backup_manager_test;
+    METRIC_VAR_DECLARE_gauge_int64(backup_running_count);
+    METRIC_VAR_DECLARE_gauge_int64(backup_max_duration_ms);
+    METRIC_VAR_DECLARE_gauge_int64(backup_file_upload_max_bytes);
 };
 
 } // namespace replication
diff --git a/src/replica/replica.cpp b/src/replica/replica.cpp
index a4d2068b2..604c179bb 100644
--- a/src/replica/replica.cpp
+++ b/src/replica/replica.cpp
@@ -209,6 +209,41 @@ METRIC_DEFINE_counter(replica,
                       dsn::metric_unit::kRequests,
                       "The number of write requests whose size exceeds threshold");
 
+METRIC_DEFINE_counter(replica,
+                      backup_started_count,
+                      dsn::metric_unit::kBackups,
+                      "The number of started backups");
+
+METRIC_DEFINE_counter(replica,
+                      backup_failed_count,
+                      dsn::metric_unit::kBackups,
+                      "The number of failed backups");
+
+METRIC_DEFINE_counter(replica,
+                      backup_successful_count,
+                      dsn::metric_unit::kBackups,
+                      "The number of successful backups");
+
+METRIC_DEFINE_counter(replica,
+                      backup_cancelled_count,
+                      dsn::metric_unit::kBackups,
+                      "The number of cancelled backups");
+
+METRIC_DEFINE_counter(replica,
+                      backup_file_upload_failed_count,
+                      dsn::metric_unit::kFileUploads,
+                      "The number of failed file uploads for backups");
+
+METRIC_DEFINE_counter(replica,
+                      backup_file_upload_successful_count,
+                      dsn::metric_unit::kFileUploads,
+                      "The number of successful file uploads for backups");
+
+METRIC_DEFINE_counter(replica,
+                      backup_file_upload_total_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The total size of uploaded files for backups");
+
 namespace dsn {
 namespace replication {
 
@@ -250,9 +285,6 @@ replica::replica(replica_stub *stub,
       _app_info(app),
       _primary_states(gpid, FLAGS_staleness_for_commit, FLAGS_batch_write_disabled),
       _potential_secondary_states(this),
-      _cold_backup_running_count(0),
-      _cold_backup_max_duration_time_ms(0),
-      _cold_backup_max_upload_file_size(0),
       _chkpt_total_size(0),
       _cur_download_size(0),
       _restore_progress(0),
@@ -288,7 +320,14 @@ replica::replica(replica_stub *stub,
       METRIC_VAR_INIT_replica(prepare_failed_requests),
       METRIC_VAR_INIT_replica(group_check_failed_requests),
       METRIC_VAR_INIT_replica(emergency_checkpoints),
-      METRIC_VAR_INIT_replica(write_size_exceed_threshold_requests)
+      METRIC_VAR_INIT_replica(write_size_exceed_threshold_requests),
+      METRIC_VAR_INIT_replica(backup_started_count),
+      METRIC_VAR_INIT_replica(backup_failed_count),
+      METRIC_VAR_INIT_replica(backup_successful_count),
+      METRIC_VAR_INIT_replica(backup_cancelled_count),
+      METRIC_VAR_INIT_replica(backup_file_upload_failed_count),
+      METRIC_VAR_INIT_replica(backup_file_upload_successful_count),
+      METRIC_VAR_INIT_replica(backup_file_upload_total_bytes)
 {
     CHECK(!_app_info.app_type.empty(), "");
     CHECK_NOTNULL(stub, "");
diff --git a/src/replica/replica.h b/src/replica/replica.h
index 5a7b2b7d5..21d216d5b 100644
--- a/src/replica/replica.h
+++ b/src/replica/replica.h
@@ -297,6 +297,12 @@ public:
 
     METRIC_DEFINE_VALUE(write_size_exceed_threshold_requests, int64_t)
     void METRIC_FUNC_NAME_SET(dup_pending_mutations)();
+    METRIC_DEFINE_INCREMENT(backup_failed_count)
+    METRIC_DEFINE_INCREMENT(backup_successful_count)
+    METRIC_DEFINE_INCREMENT(backup_cancelled_count)
+    METRIC_DEFINE_INCREMENT(backup_file_upload_failed_count)
+    METRIC_DEFINE_INCREMENT(backup_file_upload_successful_count)
+    METRIC_DEFINE_INCREMENT_BY(backup_file_upload_total_bytes)
 
     static const std::string kAppInfo;
 
@@ -612,11 +618,6 @@ private:
     std::map<std::string, cold_backup_context_ptr> _cold_backup_contexts;
     partition_split_context _split_states;
 
-    // timer task that running in replication-thread
-    std::atomic<uint64_t> _cold_backup_running_count;
-    std::atomic<uint64_t> _cold_backup_max_duration_time_ms;
-    std::atomic<uint64_t> _cold_backup_max_upload_file_size;
-
     // record the progress of restore
     int64_t _chkpt_total_size;
     std::atomic<int64_t> _cur_download_size;
@@ -696,6 +697,14 @@ private:
 
     METRIC_VAR_DECLARE_counter(write_size_exceed_threshold_requests);
 
+    METRIC_VAR_DECLARE_counter(backup_started_count);
+    METRIC_VAR_DECLARE_counter(backup_failed_count);
+    METRIC_VAR_DECLARE_counter(backup_successful_count);
+    METRIC_VAR_DECLARE_counter(backup_cancelled_count);
+    METRIC_VAR_DECLARE_counter(backup_file_upload_failed_count);
+    METRIC_VAR_DECLARE_counter(backup_file_upload_successful_count);
+    METRIC_VAR_DECLARE_counter(backup_file_upload_total_bytes);
+
     dsn::task_tracker _tracker;
     // the thread access checker
     dsn::thread_access_checker _checker;
diff --git a/src/replica/replica_backup.cpp b/src/replica/replica_backup.cpp
index 50a4b45f2..09c407929 100644
--- a/src/replica/replica_backup.cpp
+++ b/src/replica/replica_backup.cpp
@@ -42,8 +42,6 @@
 #include "common/replication_other_types.h"
 #include "dsn.layer2_types.h"
 #include "metadata_types.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica.h"
 #include "replica/replica_context.h"
 #include "replica/replication_app_base.h"
@@ -55,6 +53,7 @@
 #include "utils/filesystem.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/strings.h"
 #include "utils/thread_access_checker.h"
 #include "utils/time_utils.h"
@@ -175,7 +174,7 @@ void replica::on_cold_backup(const backup_request &request, /*out*/ backup_respo
                 backup_context->start_check();
                 backup_context->complete_check(false);
                 if (backup_context->start_checkpoint()) {
-                    _stub->_counter_cold_backup_recent_start_count->increment();
+                    METRIC_VAR_INCREMENT(backup_started_count);
                     tasking::enqueue(
                         LPC_BACKGROUND_COLD_BACKUP, &_tracker, [this, backup_context]() {
                             generate_backup_checkpoint(backup_context);
@@ -196,7 +195,7 @@ void replica::on_cold_backup(const backup_request &request, /*out*/ backup_respo
                      backup_context->progress());
             response.err = ERR_BUSY;
         } else if (backup_status == ColdBackupInvalid && backup_context->start_check()) {
-            _stub->_counter_cold_backup_recent_start_count->increment();
+            METRIC_VAR_INCREMENT(backup_started_count);
             LOG_INFO("{}: start checking backup on remote, response ERR_BUSY",
                      backup_context->name);
             tasking::enqueue(LPC_BACKGROUND_COLD_BACKUP, nullptr, [backup_context]() {
diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp
index 5eb6faa16..1323ddf0b 100644
--- a/src/replica/replica_stub.cpp
+++ b/src/replica/replica_stub.cpp
@@ -164,10 +164,10 @@ METRIC_DEFINE_gauge_int64(server,
                           "The number of origin replica dirs (*.ori) for disk migration");
 
 #ifdef DSN_ENABLE_GPERF
-METRIC_DEFINE_gauge_int64(server,
-                          tcmalloc_released_bytes,
-                          dsn::metric_unit::kBytes,
-                          "The memory bytes that are released by tcmalloc recently");
+METRIC_DEFINE_counter(server,
+                      tcmalloc_released_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The memory bytes that are released accumulatively by tcmalloc");
 #endif
 
 METRIC_DEFINE_counter(server,
@@ -325,63 +325,6 @@ replica_stub::~replica_stub(void) { close(); }
 
 void replica_stub::install_perf_counters()
 {
-    // <- Cold Backup Metrics ->
-
-    _counter_cold_backup_running_count.init_app_counter("eon.replica_stub",
-                                                        "cold.backup.running.count",
-                                                        COUNTER_TYPE_NUMBER,
-                                                        "current cold backup count");
-    _counter_cold_backup_recent_start_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.start.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup start count in the recent period");
-    _counter_cold_backup_recent_succ_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.succ.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup succeed count in the recent period");
-    _counter_cold_backup_recent_fail_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.fail.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup fail count in the recent period");
-    _counter_cold_backup_recent_cancel_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.cancel.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup cancel count in the recent period");
-    _counter_cold_backup_recent_pause_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.pause.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup pause count in the recent period");
-    _counter_cold_backup_recent_upload_file_succ_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.upload.file.succ.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup upload file succeed count in the recent period");
-    _counter_cold_backup_recent_upload_file_fail_count.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.upload.file.fail.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup upload file failed count in the recent period");
-    _counter_cold_backup_recent_upload_file_size.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.recent.upload.file.size",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current cold backup upload file size in the recent perriod");
-    _counter_cold_backup_max_duration_time_ms.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.max.duration.time.ms",
-        COUNTER_TYPE_NUMBER,
-        "current cold backup max duration time");
-    _counter_cold_backup_max_upload_file_size.init_app_counter(
-        "eon.replica_stub",
-        "cold.backup.max.upload.file.size",
-        COUNTER_TYPE_NUMBER,
-        "current cold backup max upload file size");
-
     // <- Bulk Load Metrics ->
 
     _counter_bulk_load_running_count.init_app_counter("eon.replica_stub",
@@ -1837,9 +1780,6 @@ void replica_stub::on_gc()
     uint64_t learning_count = 0;
     uint64_t learning_max_duration_time_ms = 0;
     uint64_t learning_max_copy_file_size = 0;
-    uint64_t cold_backup_running_count = 0;
-    uint64_t cold_backup_max_duration_time_ms = 0;
-    uint64_t cold_backup_max_upload_file_size = 0;
     uint64_t bulk_load_running_count = 0;
     uint64_t bulk_load_max_ingestion_time_ms = 0;
     uint64_t bulk_load_max_duration_time_ms = 0;
@@ -1859,12 +1799,6 @@ void replica_stub::on_gc()
         }
         if (rep->status() == partition_status::PS_PRIMARY ||
             rep->status() == partition_status::PS_SECONDARY) {
-            cold_backup_running_count += rep->_cold_backup_running_count.load();
-            cold_backup_max_duration_time_ms = std::max(
-                cold_backup_max_duration_time_ms, rep->_cold_backup_max_duration_time_ms.load());
-            cold_backup_max_upload_file_size = std::max(
-                cold_backup_max_upload_file_size, rep->_cold_backup_max_upload_file_size.load());
-
             if (rep->get_bulk_loader()->get_bulk_load_status() != bulk_load_status::BLS_INVALID) {
                 bulk_load_running_count++;
                 bulk_load_max_ingestion_time_ms =
@@ -1888,9 +1822,6 @@ void replica_stub::on_gc()
     METRIC_VAR_SET(learning_replicas, learning_count);
     METRIC_VAR_SET(learning_replicas_max_duration_ms, learning_max_duration_time_ms);
     METRIC_VAR_SET(learning_replicas_max_copy_file_bytes, learning_max_copy_file_size);
-    _counter_cold_backup_running_count->set(cold_backup_running_count);
-    _counter_cold_backup_max_duration_time_ms->set(cold_backup_max_duration_time_ms);
-    _counter_cold_backup_max_upload_file_size->set(cold_backup_max_upload_file_size);
     _counter_bulk_load_running_count->set(bulk_load_running_count);
     _counter_bulk_load_max_ingestion_time_ms->set(bulk_load_max_ingestion_time_ms);
     _counter_bulk_load_max_duration_time_ms->set(bulk_load_max_duration_time_ms);
@@ -2804,43 +2735,48 @@ static int64_t get_tcmalloc_numeric_property(const char *prop)
 
 uint64_t replica_stub::gc_tcmalloc_memory(bool release_all)
 {
-    auto tcmalloc_released_bytes = 0;
     if (!_release_tcmalloc_memory) {
         _is_releasing_memory.store(false);
-        METRIC_VAR_SET(tcmalloc_released_bytes, tcmalloc_released_bytes);
-        return tcmalloc_released_bytes;
+        return 0;
     }
 
     if (_is_releasing_memory.load()) {
         LOG_WARNING("This node is releasing memory...");
-        return tcmalloc_released_bytes;
+        return 0;
     }
 
     _is_releasing_memory.store(true);
+
     int64_t total_allocated_bytes =
         get_tcmalloc_numeric_property("generic.current_allocated_bytes");
     int64_t reserved_bytes = get_tcmalloc_numeric_property("tcmalloc.pageheap_free_bytes");
     if (total_allocated_bytes == -1 || reserved_bytes == -1) {
-        return tcmalloc_released_bytes;
+        return 0;
     }
 
     int64_t max_reserved_bytes =
         release_all ? 0
                     : (total_allocated_bytes * _mem_release_max_reserved_mem_percentage / 100.0);
-    if (reserved_bytes > max_reserved_bytes) {
-        int64_t release_bytes = reserved_bytes - max_reserved_bytes;
-        tcmalloc_released_bytes = release_bytes;
-        LOG_INFO("Memory release started, almost {} bytes will be released", release_bytes);
-        while (release_bytes > 0) {
-            // tcmalloc releasing memory will lock page heap, release 1MB at a time to avoid locking
-            // page heap for long time
-            ::MallocExtension::instance()->ReleaseToSystem(1024 * 1024);
-            release_bytes -= 1024 * 1024;
-        }
+    if (reserved_bytes <= max_reserved_bytes) {
+        return 0;
+    }
+
+    const int64_t expected_released_bytes = reserved_bytes - max_reserved_bytes;
+    LOG_INFO("Memory release started, almost {} bytes will be released", expected_released_bytes);
+
+    int64_t unreleased_bytes = expected_released_bytes;
+    while (unreleased_bytes > 0) {
+        // tcmalloc releasing memory will lock page heap, release 1MB at a time to avoid locking
+        // page heap for long time
+        static const int64_t kReleasedBytesEachTime = 1024 * 1024;
+        ::MallocExtension::instance()->ReleaseToSystem(kReleasedBytesEachTime);
+        unreleased_bytes -= kReleasedBytesEachTime;
     }
-    METRIC_VAR_SET(tcmalloc_released_bytes, tcmalloc_released_bytes);
+    METRIC_VAR_INCREMENT_BY(tcmalloc_released_bytes, expected_released_bytes);
+
     _is_releasing_memory.store(false);
-    return tcmalloc_released_bytes;
+
+    return expected_released_bytes;
 }
 #endif
 
diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h
index 10e964986..a1087204c 100644
--- a/src/replica/replica_stub.h
+++ b/src/replica/replica_stub.h
@@ -515,7 +515,7 @@ private:
     METRIC_VAR_DECLARE_gauge_int64(replica_origin_dirs);
 
 #ifdef DSN_ENABLE_GPERF
-    METRIC_VAR_DECLARE_gauge_int64(tcmalloc_released_bytes);
+    METRIC_VAR_DECLARE_counter(tcmalloc_released_bytes);
 #endif
 
     METRIC_VAR_DECLARE_counter(read_failed_requests);
@@ -523,18 +523,6 @@ private:
     METRIC_VAR_DECLARE_counter(read_busy_requests);
     METRIC_VAR_DECLARE_counter(write_busy_requests);
 
-    perf_counter_wrapper _counter_cold_backup_running_count;
-    perf_counter_wrapper _counter_cold_backup_recent_start_count;
-    perf_counter_wrapper _counter_cold_backup_recent_succ_count;
-    perf_counter_wrapper _counter_cold_backup_recent_fail_count;
-    perf_counter_wrapper _counter_cold_backup_recent_cancel_count;
-    perf_counter_wrapper _counter_cold_backup_recent_pause_count;
-    perf_counter_wrapper _counter_cold_backup_recent_upload_file_succ_count;
-    perf_counter_wrapper _counter_cold_backup_recent_upload_file_fail_count;
-    perf_counter_wrapper _counter_cold_backup_recent_upload_file_size;
-    perf_counter_wrapper _counter_cold_backup_max_duration_time_ms;
-    perf_counter_wrapper _counter_cold_backup_max_upload_file_size;
-
     // <- Bulk load Metrics ->
     perf_counter_wrapper _counter_bulk_load_running_count;
     perf_counter_wrapper _counter_bulk_load_downloading_count;
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index e90d5d9af..78c067ea3 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -684,6 +684,8 @@ enum class metric_unit : size_t
     kLearns,
     kRounds,
     kResets,
+    kBackups,
+    kFileUploads,
     kInvalidUnit,
 };
 


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