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/05 07:27:55 UTC

[incubator-pegasus] branch migrate-metrics-dev updated (77ed557be -> 72f6e3836)

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

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


 discard 77ed557be feat(new_metrics): migrate metrics for replica_stub (part 4) (#1463)
    omit 746932310 feat(new_metrics): migrate metrics for replica_stub (part 3) (#1462)
    omit e337fa881 feat(collector): migrate the collector from pegasus-kv/collector (#1461)
    omit e73a889ea feat(new_metrics): migrate metrics for replica_stub (part 2) (#1459)
    omit 665d53bc5 feat(new_metrics): migrate metrics for replica_stub (part 1) (#1455)
    omit 6b8ac2058 feat(new_metrics): migrate replica-level metrics for pegasus_manual_compact_service (#1443)
    omit 0ff94b984 feat(new_metrics): migrate partition-level metrics for partition_guardian (#1440)
    omit aead1b9bf feat(new_metrics): add backup-policy-level metric entity and migrate backup-policy-level metrics for meta_backup_service (#1438)
    omit 53cca5623 feat(new_metrics): migrate server-level metrics for meta_service (#1437)
    omit 81afa0602 feat(new_metrics): add partition-level metric entity and migrate partition-level metrics for greedy_load_balancer of meta (#1435)
    omit 280541712 feat(new_metrics): add table-level metric entity and migrate table-level metrics for server_state of meta (#1431)
    omit 0ae934397 feat(new_metrics): add disk-level metric entity and migrate disk-level metrics for fs_manager (#1427)
    omit 6ffd53e56 feat(new_metrics): migrate server-level metrics for nfs (#1421)
    omit eb3a781a7 feat(new_metrics): migrate built-in server-level metrics (#1418)
    omit 4cfbced69 feat(new_metrics): add server-level metric entity (#1415)
    omit 8986ffb29 feat(new_metrics): migrate replica-level metrics for pegasus_mutation_duplicator (#1413)
    omit 94e522e1d feat(new_metrics): migrate replica-level metrics for pegasus_event_listener (#1407)
    omit 2ee9c545a feat(new_metrics): migrate replica-level metrics for replica class (#1397)
    omit a023b82c6 feat(new_metrics): migrate replica-level metrics for capacity_unit_calculator (#1387)
    omit 64d930590 feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 2) (#1386)
    omit 7e30a4ccc feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 1) (#1374)
    omit 39c0f8f5e feat(new_metrics): migrate replica-level metrics for write service (#1351)
    omit db099e41a feat(new_metrics): add replica-level metric entity (#1345)
     add e655400af feat(FQDN): Implemention of struct host_port_group (#1436)
     add d37fca91f fix(log): fix using of macro LOG_AND_RETURN_NOT_* print duplicate prefixes (#1465)
     new 09b5d5926 feat(new_metrics): add replica-level metric entity (#1345)
     new bcec273f8 feat(new_metrics): migrate replica-level metrics for write service (#1351)
     new 624ed5c19 feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 1) (#1374)
     new ecce3000c feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 2) (#1386)
     new 8d47b5098 feat(new_metrics): migrate replica-level metrics for capacity_unit_calculator (#1387)
     new de1b95f40 feat(new_metrics): migrate replica-level metrics for replica class (#1397)
     new da8bf31ed feat(new_metrics): migrate replica-level metrics for pegasus_event_listener (#1407)
     new b9c541ffe feat(new_metrics): migrate replica-level metrics for pegasus_mutation_duplicator (#1413)
     new 20b88806c feat(new_metrics): add server-level metric entity (#1415)
     new b01afb395 feat(new_metrics): migrate built-in server-level metrics (#1418)
     new 990da555b feat(new_metrics): migrate server-level metrics for nfs (#1421)
     new 8a8dbc24f feat(new_metrics): add disk-level metric entity and migrate disk-level metrics for fs_manager (#1427)
     new db1d82f58 feat(new_metrics): add table-level metric entity and migrate table-level metrics for server_state of meta (#1431)
     new 69ca0ec97 feat(new_metrics): add partition-level metric entity and migrate partition-level metrics for greedy_load_balancer of meta (#1435)
     new 5325b7654 feat(new_metrics): migrate server-level metrics for meta_service (#1437)
     new e83402be6 feat(new_metrics): add backup-policy-level metric entity and migrate backup-policy-level metrics for meta_backup_service (#1438)
     new af3cad27d feat(new_metrics): migrate partition-level metrics for partition_guardian (#1440)
     new 142c44c08 feat(new_metrics): migrate replica-level metrics for pegasus_manual_compact_service (#1443)
     new eccc3be60 feat(new_metrics): migrate metrics for replica_stub (part 1) (#1455)
     new 0d46760c6 feat(new_metrics): migrate metrics for replica_stub (part 2) (#1459)
     new ddd3a2c12 feat(collector): migrate the collector from pegasus-kv/collector (#1461)
     new b99bd0002 feat(new_metrics): migrate metrics for replica_stub (part 3) (#1462)
     new 72f6e3836 feat(new_metrics): migrate metrics for replica_stub (part 4) (#1463)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (77ed557be)
            \
             N -- N -- N   refs/heads/migrate-metrics-dev (72f6e3836)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 23 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/replica/replication_app_base.cpp |  24 ++--
 src/runtime/rpc/group_host_port.h    | 257 +++++++++++++++++++++++++++++++++++
 src/runtime/rpc/rpc_host_port.cpp    |  25 +++-
 src/runtime/rpc/rpc_host_port.h      |  15 +-
 src/runtime/test/host_port_test.cpp  |  71 +++++++++-
 5 files changed, 369 insertions(+), 23 deletions(-)
 create mode 100644 src/runtime/rpc/group_host_port.h


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


[incubator-pegasus] 22/23: feat(new_metrics): migrate metrics for replica_stub (part 3) (#1462)

Posted by wa...@apache.org.
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 b99bd00028216ca0711d6d536d7ad6e8f7e60cc1
Author: Dan Wang <wa...@apache.org>
AuthorDate: Thu Apr 27 11:02:54 2023 +0800

    feat(new_metrics): migrate metrics for replica_stub (part 3) (#1462)
    
    https://github.com/apache/incubator-pegasus/issues/1454
    
    This is the 3rd part of migrating metrics of replica_stub to new framework.
    
    During this migration, there are 3 metrics which are changed from server-level
    to replica-level, including the number of failed RPC_PREPARE requests, the
    number of failed RPC_GROUP_CHECK requests launched by primary replicas,
    the number of triggered emergency checkpoints.
    
    Another 7 metrics are still kept server-level, the number of replicas whose
    dirs are moved as error or garbage, the number of removed replica dirs, error
    replica dirs (*.err), garbage replica dirs (*.gar), tmp replica dirs (*.tmp) and origin
    replica dirs (*.ori) for disk migration.
    
    There are 2 metrics removed, since both are shared-log-related.
---
 src/nfs/nfs_client_impl.cpp                |   6 +-
 src/nfs/nfs_client_impl.h                  |   2 +-
 src/nfs/nfs_server_impl.cpp                |   6 +-
 src/nfs/nfs_server_impl.h                  |   2 +-
 src/replica/replica.cpp                    |  20 ++++-
 src/replica/replica.h                      |   6 ++
 src/replica/replica_2pc.cpp                |   2 +-
 src/replica/replica_check.cpp              |   5 +-
 src/replica/replica_chkpt.cpp              |   7 +-
 src/replica/replica_learn.cpp              |  12 ---
 src/replica/replica_stub.cpp               | 136 ++++++++++++-----------------
 src/replica/replica_stub.h                 |  21 ++---
 src/server/pegasus_mutation_duplicator.cpp |  12 +--
 src/server/pegasus_mutation_duplicator.h   |   4 +-
 src/utils/metrics.h                        |   4 +-
 15 files changed, 113 insertions(+), 132 deletions(-)

diff --git a/src/nfs/nfs_client_impl.cpp b/src/nfs/nfs_client_impl.cpp
index 8ddadc46c..4c10b4da8 100644
--- a/src/nfs/nfs_client_impl.cpp
+++ b/src/nfs/nfs_client_impl.cpp
@@ -48,7 +48,7 @@ METRIC_DEFINE_counter(server,
                       "The accumulated data size in bytes requested by client during nfs copy");
 
 METRIC_DEFINE_counter(server,
-                      nfs_client_failed_copy_requests,
+                      nfs_client_copy_failed_requests,
                       dsn::metric_unit::kRequests,
                       "The number of failed nfs copy requests (requested by client)");
 
@@ -121,7 +121,7 @@ nfs_client_impl::nfs_client_impl()
       _copy_requests_low(FLAGS_max_file_copy_request_count_per_file),
       _high_priority_remaining_time(FLAGS_high_priority_speed_rate),
       METRIC_VAR_INIT_server(nfs_client_copy_bytes),
-      METRIC_VAR_INIT_server(nfs_client_failed_copy_requests),
+      METRIC_VAR_INIT_server(nfs_client_copy_failed_requests),
       METRIC_VAR_INIT_server(nfs_client_write_bytes),
       METRIC_VAR_INIT_server(nfs_client_failed_writes)
 {
@@ -345,7 +345,7 @@ void nfs_client_impl::end_copy(::dsn::error_code err,
     }
 
     if (err != ::dsn::ERR_OK) {
-        METRIC_VAR_INCREMENT(nfs_client_failed_copy_requests);
+        METRIC_VAR_INCREMENT(nfs_client_copy_failed_requests);
 
         if (!fc->user_req->is_finished) {
             if (reqc->retry_count > 0) {
diff --git a/src/nfs/nfs_client_impl.h b/src/nfs/nfs_client_impl.h
index 0c15fc8b3..183ac38a9 100644
--- a/src/nfs/nfs_client_impl.h
+++ b/src/nfs/nfs_client_impl.h
@@ -312,7 +312,7 @@ private:
     std::deque<copy_request_ex_ptr> _local_writes;
 
     METRIC_VAR_DECLARE_counter(nfs_client_copy_bytes);
-    METRIC_VAR_DECLARE_counter(nfs_client_failed_copy_requests);
+    METRIC_VAR_DECLARE_counter(nfs_client_copy_failed_requests);
     METRIC_VAR_DECLARE_counter(nfs_client_write_bytes);
     METRIC_VAR_DECLARE_counter(nfs_client_failed_writes);
 
diff --git a/src/nfs/nfs_server_impl.cpp b/src/nfs/nfs_server_impl.cpp
index 25632d4f9..ac2d6a14d 100644
--- a/src/nfs/nfs_server_impl.cpp
+++ b/src/nfs/nfs_server_impl.cpp
@@ -55,7 +55,7 @@ METRIC_DEFINE_counter(
 
 METRIC_DEFINE_counter(
     server,
-    nfs_server_failed_copy_requests,
+    nfs_server_copy_failed_requests,
     dsn::metric_unit::kRequests,
     "The number of nfs copy requests (received by server) that fail to read local file in server");
 
@@ -77,7 +77,7 @@ DSN_DECLARE_int32(file_close_expire_time_ms);
 nfs_service_impl::nfs_service_impl()
     : ::dsn::serverlet<nfs_service_impl>("nfs"),
       METRIC_VAR_INIT_server(nfs_server_copy_bytes),
-      METRIC_VAR_INIT_server(nfs_server_failed_copy_requests)
+      METRIC_VAR_INIT_server(nfs_server_copy_failed_requests)
 {
     _file_close_timer = ::dsn::tasking::enqueue_timer(
         LPC_NFS_FILE_CLOSE_TIMER,
@@ -167,7 +167,7 @@ void nfs_service_impl::internal_read_callback(error_code err, size_t sz, callbac
 
     if (err != ERR_OK) {
         LOG_ERROR("[nfs_service] read file {} failed, err = {}", cp.file_path, err);
-        METRIC_VAR_INCREMENT(nfs_server_failed_copy_requests);
+        METRIC_VAR_INCREMENT(nfs_server_copy_failed_requests);
     } else {
         METRIC_VAR_INCREMENT_BY(nfs_server_copy_bytes, sz);
     }
diff --git a/src/nfs/nfs_server_impl.h b/src/nfs/nfs_server_impl.h
index 4c07a4996..4a4c5b5c4 100644
--- a/src/nfs/nfs_server_impl.h
+++ b/src/nfs/nfs_server_impl.h
@@ -138,7 +138,7 @@ private:
         _send_token_buckets; // rate limiter of send to remote
 
     METRIC_VAR_DECLARE_counter(nfs_server_copy_bytes);
-    METRIC_VAR_DECLARE_counter(nfs_server_failed_copy_requests);
+    METRIC_VAR_DECLARE_counter(nfs_server_copy_failed_requests);
 
     std::unique_ptr<command_deregister> _nfs_max_send_rate_megabytes_cmd;
 
diff --git a/src/replica/replica.cpp b/src/replica/replica.cpp
index 349bae53d..a660ab509 100644
--- a/src/replica/replica.cpp
+++ b/src/replica/replica.cpp
@@ -189,6 +189,21 @@ METRIC_DEFINE_counter(replica,
                       dsn::metric_unit::kLearns,
                       "The number of successful learns launched by learner");
 
+METRIC_DEFINE_counter(replica,
+                      prepare_failed_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of failed RPC_PREPARE requests");
+
+METRIC_DEFINE_counter(replica,
+                      group_check_failed_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of failed RPC_GROUP_CHECK requests launched by primary replicas");
+
+METRIC_DEFINE_counter(replica,
+                      emergency_checkpoints,
+                      dsn::metric_unit::kCheckpoints,
+                      "The number of triggered emergency checkpoints");
+
 namespace dsn {
 namespace replication {
 
@@ -264,7 +279,10 @@ replica::replica(replica_stub *stub,
       METRIC_VAR_INIT_replica(learn_lt_log_responses),
       METRIC_VAR_INIT_replica(learn_resets),
       METRIC_VAR_INIT_replica(learn_failed_count),
-      METRIC_VAR_INIT_replica(learn_successful_count)
+      METRIC_VAR_INIT_replica(learn_successful_count),
+      METRIC_VAR_INIT_replica(prepare_failed_requests),
+      METRIC_VAR_INIT_replica(group_check_failed_requests),
+      METRIC_VAR_INIT_replica(emergency_checkpoints)
 {
     CHECK(!_app_info.app_type.empty(), "");
     CHECK_NOTNULL(stub, "");
diff --git a/src/replica/replica.h b/src/replica/replica.h
index 25494fd2c..8feefb109 100644
--- a/src/replica/replica.h
+++ b/src/replica/replica.h
@@ -676,6 +676,12 @@ private:
     METRIC_VAR_DECLARE_counter(learn_failed_count);
     METRIC_VAR_DECLARE_counter(learn_successful_count);
 
+    METRIC_VAR_DECLARE_counter(prepare_failed_requests);
+
+    METRIC_VAR_DECLARE_counter(group_check_failed_requests);
+
+    METRIC_VAR_DECLARE_counter(emergency_checkpoints);
+
     dsn::task_tracker _tracker;
     // the thread access checker
     dsn::thread_access_checker _checker;
diff --git a/src/replica/replica_2pc.cpp b/src/replica/replica_2pc.cpp
index 9525466b2..970df6688 100644
--- a/src/replica/replica_2pc.cpp
+++ b/src/replica/replica_2pc.cpp
@@ -763,7 +763,7 @@ void replica::on_prepare_reply(std::pair<mutation_ptr, partition_status::type> p
             }
         }
 
-        _stub->_counter_replicas_recent_prepare_fail_count->increment();
+        METRIC_VAR_INCREMENT(prepare_failed_requests);
 
         // make sure this is before any later commit ops
         // because now commit ops may lead to new prepare ops
diff --git a/src/replica/replica_check.cpp b/src/replica/replica_check.cpp
index 8cd0325e2..59aece790 100644
--- a/src/replica/replica_check.cpp
+++ b/src/replica/replica_check.cpp
@@ -48,8 +48,6 @@
 #include "duplication/replica_duplicator_manager.h"
 #include "metadata_types.h"
 #include "mutation.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica.h"
 #include "replica/prepare_list.h"
 #include "replica/replica_context.h"
@@ -65,6 +63,7 @@
 #include "utils/fail_point.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/string_view.h"
 #include "utils/thread_access_checker.h"
 
@@ -253,7 +252,7 @@ void replica::on_group_check_reply(error_code err,
             err = resp->err;
         }
         handle_remote_failure(req->config.status, req->node, err, "group check");
-        _stub->_counter_replicas_recent_group_check_fail_count->increment();
+        METRIC_VAR_INCREMENT(group_check_failed_requests);
     } else {
         if (resp->learner_status_ == learner_status::LearningSucceeded &&
             req->config.status == partition_status::PS_POTENTIAL_SECONDARY) {
diff --git a/src/replica/replica_chkpt.cpp b/src/replica/replica_chkpt.cpp
index 5985e5d8d..7c9f6f931 100644
--- a/src/replica/replica_chkpt.cpp
+++ b/src/replica/replica_chkpt.cpp
@@ -50,8 +50,6 @@
 #include "duplication/replica_duplicator_manager.h"
 #include "metadata_types.h"
 #include "mutation_log.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica.h"
 #include "replica/prepare_list.h"
 #include "replica/replica_context.h"
@@ -240,8 +238,9 @@ void replica::init_checkpoint(bool is_emergency)
                      0,
                      10_ms);
 
-    if (is_emergency)
-        _stub->_counter_recent_trigger_emergency_checkpoint_count->increment();
+    if (is_emergency) {
+        METRIC_VAR_INCREMENT(emergency_checkpoints);
+    }
 }
 
 // ThreadPool: THREAD_POOL_REPLICATION
diff --git a/src/replica/replica_learn.cpp b/src/replica/replica_learn.cpp
index b123fbe83..193ace628 100644
--- a/src/replica/replica_learn.cpp
+++ b/src/replica/replica_learn.cpp
@@ -80,18 +80,6 @@
 #include "utils/metrics.h"
 #include "utils/thread_access_checker.h"
 
-METRIC_DECLARE_counter(learn_count);
-METRIC_DECLARE_counter(learn_rounds);
-METRIC_DECLARE_counter(learn_copy_files);
-METRIC_DECLARE_counter(learn_copy_file_bytes);
-METRIC_DECLARE_counter(learn_copy_buffer_bytes);
-METRIC_DECLARE_counter(learn_lt_cache_responses);
-METRIC_DECLARE_counter(learn_lt_app_responses);
-METRIC_DECLARE_counter(learn_lt_log_responses);
-METRIC_DECLARE_counter(learn_resets);
-METRIC_DECLARE_counter(learn_failed_count);
-METRIC_DECLARE_counter(learn_successful_count);
-
 namespace dsn {
 namespace replication {
 
diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp
index 35c0b742d..48718952c 100644
--- a/src/replica/replica_stub.cpp
+++ b/src/replica/replica_stub.cpp
@@ -129,6 +129,41 @@ METRIC_DEFINE_gauge_int64(
     dsn::metric_unit::kBytes,
     "The max size of files that are copied from learnee among all learning replicas");
 
+METRIC_DEFINE_counter(server,
+                      moved_error_replicas,
+                      dsn::metric_unit::kReplicas,
+                      "The number of replicas whose dirs are moved as error");
+
+METRIC_DEFINE_counter(server,
+                      moved_garbage_replicas,
+                      dsn::metric_unit::kReplicas,
+                      "The number of replicas whose dirs are moved as garbage");
+
+METRIC_DEFINE_counter(server,
+                      replica_removed_dirs,
+                      dsn::metric_unit::kDirs,
+                      "The number of removed replica dirs");
+
+METRIC_DEFINE_gauge_int64(server,
+                          replica_error_dirs,
+                          dsn::metric_unit::kDirs,
+                          "The number of error replica dirs (*.err)");
+
+METRIC_DEFINE_gauge_int64(server,
+                          replica_garbage_dirs,
+                          dsn::metric_unit::kDirs,
+                          "The number of garbage replica dirs (*.gar)");
+
+METRIC_DEFINE_gauge_int64(server,
+                          replica_tmp_dirs,
+                          dsn::metric_unit::kDirs,
+                          "The number of tmp replica dirs (*.tmp) for disk migration");
+
+METRIC_DEFINE_gauge_int64(server,
+                          replica_origin_dirs,
+                          dsn::metric_unit::kDirs,
+                          "The number of origin replica dirs (*.ori) for disk migration");
+
 namespace dsn {
 namespace replication {
 
@@ -237,7 +272,14 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/,
       METRIC_VAR_INIT_server(closing_replicas),
       METRIC_VAR_INIT_server(learning_replicas),
       METRIC_VAR_INIT_server(learning_replicas_max_duration_ms),
-      METRIC_VAR_INIT_server(learning_replicas_max_copy_file_bytes)
+      METRIC_VAR_INIT_server(learning_replicas_max_copy_file_bytes),
+      METRIC_VAR_INIT_server(moved_error_replicas),
+      METRIC_VAR_INIT_server(moved_garbage_replicas),
+      METRIC_VAR_INIT_server(replica_removed_dirs),
+      METRIC_VAR_INIT_server(replica_error_dirs),
+      METRIC_VAR_INIT_server(replica_garbage_dirs),
+      METRIC_VAR_INIT_server(replica_tmp_dirs),
+      METRIC_VAR_INIT_server(replica_origin_dirs)
 {
 #ifdef DSN_ENABLE_GPERF
     _is_releasing_memory = false;
@@ -255,66 +297,6 @@ replica_stub::~replica_stub(void) { close(); }
 
 void replica_stub::install_perf_counters()
 {
-    _counter_replicas_recent_prepare_fail_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.recent.prepare.fail.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "prepare fail count in the recent period");
-    _counter_replicas_recent_replica_move_error_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.recent.replica.move.error.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "replica move to error count in the recent period");
-    _counter_replicas_recent_replica_move_garbage_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.recent.replica.move.garbage.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "replica move to garbage count in the recent period");
-    _counter_replicas_recent_replica_remove_dir_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.recent.replica.remove.dir.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "replica directory remove count in the recent period");
-    _counter_replicas_error_replica_dir_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.error.replica.dir.count",
-        COUNTER_TYPE_NUMBER,
-        "error replica directory(*.err) count");
-    _counter_replicas_garbage_replica_dir_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.garbage.replica.dir.count",
-        COUNTER_TYPE_NUMBER,
-        "garbage replica directory(*.gar) count");
-    _counter_replicas_tmp_replica_dir_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.tmp.replica.dir.count",
-        COUNTER_TYPE_NUMBER,
-        "disk migration tmp replica directory(*.tmp) count");
-    _counter_replicas_origin_replica_dir_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.origin.replica.dir.count",
-        COUNTER_TYPE_NUMBER,
-        "disk migration origin replica directory(.ori) count");
-
-    _counter_replicas_recent_group_check_fail_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.recent.group.check.fail.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "group check fail count in the recent period");
-
-    _counter_shared_log_size.init_app_counter(
-        "eon.replica_stub", "shared.log.size(MB)", COUNTER_TYPE_NUMBER, "shared log size(MB)");
-    _counter_shared_log_recent_write_size.init_app_counter(
-        "eon.replica_stub",
-        "shared.log.recent.write.size",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "shared log write size in the recent period");
-    _counter_recent_trigger_emergency_checkpoint_count.init_app_counter(
-        "eon.replica_stub",
-        "recent.trigger.emergency.checkpoint.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "trigger emergency checkpoint count in the recent period");
-
     // <- Duplication Metrics ->
 
     _counter_dup_confirmed_rate.init_app_counter("eon.replica_stub",
@@ -563,10 +545,8 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f
     _options.slog_dir = cdir;
     initialize_fs_manager(_options.data_dirs, _options.data_dir_tags);
 
-    _log = new mutation_log_shared(_options.slog_dir,
-                                   FLAGS_log_shared_file_size_mb,
-                                   FLAGS_log_shared_force_flush,
-                                   &_counter_shared_log_recent_write_size);
+    _log = new mutation_log_shared(
+        _options.slog_dir, FLAGS_log_shared_file_size_mb, FLAGS_log_shared_force_flush);
     LOG_INFO("slog_dir = {}", _options.slog_dir);
 
     // init rps
@@ -670,7 +650,7 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f
         for (auto it = rps.begin(); it != rps.end(); ++it) {
             it->second->close();
             move_to_err_path(it->second->dir(), "initialize replica");
-            _counter_replicas_recent_replica_move_error_count->increment();
+            METRIC_VAR_INCREMENT(moved_error_replicas);
         }
         rps.clear();
 
@@ -680,10 +660,8 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f
         CHECK(utils::filesystem::remove_path(_options.slog_dir),
               "remove directory {} failed",
               _options.slog_dir);
-        _log = new mutation_log_shared(_options.slog_dir,
-                                       FLAGS_log_shared_file_size_mb,
-                                       FLAGS_log_shared_force_flush,
-                                       &_counter_shared_log_recent_write_size);
+        _log = new mutation_log_shared(
+            _options.slog_dir, FLAGS_log_shared_file_size_mb, FLAGS_log_shared_force_flush);
         CHECK_EQ_MSG(_log->open(nullptr, [this](error_code err) { this->handle_log_failure(err); }),
                      ERR_OK,
                      "restart log service failed");
@@ -1756,7 +1734,7 @@ void replica_stub::on_gc_replica(replica_stub_ptr this_, gpid id)
         LOG_WARNING("gc_replica: replica_dir_op succeed to move directory '{}' to '{}'",
                     replica_path,
                     rename_path);
-        _counter_replicas_recent_replica_move_garbage_count->increment();
+        METRIC_VAR_INCREMENT(moved_garbage_replicas);
     }
 }
 
@@ -1889,8 +1867,6 @@ void replica_stub::on_gc()
                 }
             }
         }
-
-        _counter_shared_log_size->set(_log->total_size() / (1024 * 1024));
     }
 
     // statistic learning info
@@ -1973,11 +1949,11 @@ void replica_stub::on_disk_stat()
     update_disk_holding_replicas();
     update_disks_status();
 
-    _counter_replicas_error_replica_dir_count->set(report.error_replica_count);
-    _counter_replicas_garbage_replica_dir_count->set(report.garbage_replica_count);
-    _counter_replicas_tmp_replica_dir_count->set(report.disk_migrate_tmp_count);
-    _counter_replicas_origin_replica_dir_count->set(report.disk_migrate_origin_count);
-    _counter_replicas_recent_replica_remove_dir_count->add(report.remove_dir_count);
+    METRIC_VAR_SET(replica_error_dirs, report.error_replica_count);
+    METRIC_VAR_SET(replica_garbage_dirs, report.garbage_replica_count);
+    METRIC_VAR_SET(replica_tmp_dirs, report.disk_migrate_tmp_count);
+    METRIC_VAR_SET(replica_origin_dirs, report.disk_migrate_origin_count);
+    METRIC_VAR_INCREMENT_BY(replica_removed_dirs, report.remove_dir_count);
 
     LOG_INFO("finish to update disk stat, time_used_ns = {}", dsn_now_ns() - start);
 }
@@ -2279,7 +2255,7 @@ replica *replica_stub::load_replica(const char *dir)
         // clear work on failure
         if (dsn::utils::filesystem::directory_exists(dir)) {
             move_to_err_path(dir, "load replica");
-            _counter_replicas_recent_replica_move_error_count->increment();
+            METRIC_VAR_INCREMENT(moved_error_replicas);
             _fs_manager.remove_replica(pid);
         }
 
@@ -2364,7 +2340,7 @@ void replica_stub::close_replica(replica_ptr r)
     if (r->is_data_corrupted()) {
         _fs_manager.remove_replica(id);
         move_to_err_path(r->dir(), "trash replica");
-        _counter_replicas_recent_replica_move_error_count->increment();
+        METRIC_VAR_INCREMENT(moved_error_replicas);
     }
 
     LOG_INFO("{}: finish to close replica", name);
diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h
index adaef7220..7bcf4e68b 100644
--- a/src/replica/replica_stub.h
+++ b/src/replica/replica_stub.h
@@ -513,20 +513,13 @@ private:
     METRIC_VAR_DECLARE_gauge_int64(learning_replicas_max_duration_ms);
     METRIC_VAR_DECLARE_gauge_int64(learning_replicas_max_copy_file_bytes);
 
-    perf_counter_wrapper _counter_replicas_recent_prepare_fail_count;
-    perf_counter_wrapper _counter_replicas_recent_replica_move_error_count;
-    perf_counter_wrapper _counter_replicas_recent_replica_move_garbage_count;
-    perf_counter_wrapper _counter_replicas_recent_replica_remove_dir_count;
-    perf_counter_wrapper _counter_replicas_error_replica_dir_count;
-    perf_counter_wrapper _counter_replicas_garbage_replica_dir_count;
-    perf_counter_wrapper _counter_replicas_tmp_replica_dir_count;
-    perf_counter_wrapper _counter_replicas_origin_replica_dir_count;
-
-    perf_counter_wrapper _counter_replicas_recent_group_check_fail_count;
-
-    perf_counter_wrapper _counter_shared_log_size;
-    perf_counter_wrapper _counter_shared_log_recent_write_size;
-    perf_counter_wrapper _counter_recent_trigger_emergency_checkpoint_count;
+    METRIC_VAR_DECLARE_counter(moved_error_replicas);
+    METRIC_VAR_DECLARE_counter(moved_garbage_replicas);
+    METRIC_VAR_DECLARE_counter(replica_removed_dirs);
+    METRIC_VAR_DECLARE_gauge_int64(replica_error_dirs);
+    METRIC_VAR_DECLARE_gauge_int64(replica_garbage_dirs);
+    METRIC_VAR_DECLARE_gauge_int64(replica_tmp_dirs);
+    METRIC_VAR_DECLARE_gauge_int64(replica_origin_dirs);
 
     // <- Duplication Metrics ->
     // TODO(wutao1): calculate the counters independently for each remote cluster
diff --git a/src/server/pegasus_mutation_duplicator.cpp b/src/server/pegasus_mutation_duplicator.cpp
index 8d87ce60b..74832d5e6 100644
--- a/src/server/pegasus_mutation_duplicator.cpp
+++ b/src/server/pegasus_mutation_duplicator.cpp
@@ -48,12 +48,12 @@
 #include "utils/rand.h"
 
 METRIC_DEFINE_counter(replica,
-                      successful_mutation_dup_requests,
+                      mutation_dup_successful_requests,
                       dsn::metric_unit::kRequests,
                       "The number of successful DUPLICATE requests sent from mutation duplicator");
 
 METRIC_DEFINE_counter(replica,
-                      failed_mutation_dup_requests,
+                      mutation_dup_failed_requests,
                       dsn::metric_unit::kRequests,
                       "The number of failed DUPLICATE requests sent from mutation duplicator");
 
@@ -107,8 +107,8 @@ pegasus_mutation_duplicator::pegasus_mutation_duplicator(dsn::replication::repli
                                                          dsn::string_view app)
     : mutation_duplicator(r),
       _remote_cluster(remote_cluster),
-      METRIC_VAR_INIT_replica(successful_mutation_dup_requests),
-      METRIC_VAR_INIT_replica(failed_mutation_dup_requests)
+      METRIC_VAR_INIT_replica(mutation_dup_successful_requests),
+      METRIC_VAR_INIT_replica(mutation_dup_failed_requests)
 {
     // initialize pegasus-client when this class is first time used.
     static __attribute__((unused)) bool _dummy = pegasus_client_factory::initialize(nullptr);
@@ -162,7 +162,7 @@ void pegasus_mutation_duplicator::on_duplicate_reply(uint64_t hash,
     }
 
     if (perr != PERR_OK || err != dsn::ERR_OK) {
-        METRIC_VAR_INCREMENT(failed_mutation_dup_requests);
+        METRIC_VAR_INCREMENT(mutation_dup_failed_requests);
 
         // randomly log the 1% of the failed duplicate rpc, because minor number of
         // errors are acceptable.
@@ -175,7 +175,7 @@ void pegasus_mutation_duplicator::on_duplicate_reply(uint64_t hash,
         // duplicating an illegal write to server is unacceptable, fail fast.
         CHECK_NE_PREFIX_MSG(perr, PERR_INVALID_ARGUMENT, rpc.response().error_hint);
     } else {
-        METRIC_VAR_INCREMENT(successful_mutation_dup_requests);
+        METRIC_VAR_INCREMENT(mutation_dup_successful_requests);
         _total_shipped_size +=
             rpc.dsn_request()->header->body_length + rpc.dsn_request()->header->hdr_length;
     }
diff --git a/src/server/pegasus_mutation_duplicator.h b/src/server/pegasus_mutation_duplicator.h
index 9a5aa086c..dfe126df7 100644
--- a/src/server/pegasus_mutation_duplicator.h
+++ b/src/server/pegasus_mutation_duplicator.h
@@ -89,8 +89,8 @@ private:
 
     size_t _total_shipped_size{0};
 
-    METRIC_VAR_DECLARE_counter(successful_mutation_dup_requests);
-    METRIC_VAR_DECLARE_counter(failed_mutation_dup_requests);
+    METRIC_VAR_DECLARE_counter(mutation_dup_successful_requests);
+    METRIC_VAR_DECLARE_counter(mutation_dup_failed_requests);
 };
 
 // Decodes the binary `request_data` into write request in thrift struct, and
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index b5e31c050..ec2fb6977 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -652,8 +652,8 @@ enum class metric_unit : size_t
     kMegaBytes,
     kCapacityUnits,
     kPercent,
-    kPartitions,
     kReplicas,
+    kPartitions,
     kServers,
     kRequests,
     kResponses,
@@ -662,7 +662,9 @@ enum class metric_unit : size_t
     kValues,
     kKeys,
     kFiles,
+    kDirs,
     kAmplification,
+    kCheckpoints,
     kFlushes,
     kCompactions,
     kWrites,


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


[incubator-pegasus] 18/23: feat(new_metrics): migrate replica-level metrics for pegasus_manual_compact_service (#1443)

Posted by wa...@apache.org.
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 142c44c0896e391e2240e8e8b0f8c7a2cb9e5ebf
Author: Dan Wang <wa...@apache.org>
AuthorDate: Mon Apr 17 21:38:23 2023 +0800

    feat(new_metrics): migrate replica-level metrics for pegasus_manual_compact_service (#1443)
    
    https://github.com/apache/incubator-pegasus/issues/1441
    
    In perf counters, all of the 2 metrics of `pegasus_manual_compact_service`
    are about the numbers of tasks of rocksdb manual compaction: one is  the
    number of current queued tasks, while another is the number of current
    running tasks. Both metrics are server-level.
    
    They would become replica-level after migrating to the new metrics, based
    on which server-level ones could also be achieved. A convenient class
    `auto_count` is also provided to increment gauge that will be decremented
    automatically at the end of the scope.
---
 src/server/pegasus_manual_compact_service.cpp | 37 ++++++++---------
 src/server/pegasus_manual_compact_service.h   |  6 +--
 src/utils/metrics.h                           | 57 ++++++++++++++++++++++-----
 src/utils/test/metrics_test.cpp               | 39 ++++++++++++++++++
 4 files changed, 108 insertions(+), 31 deletions(-)

diff --git a/src/server/pegasus_manual_compact_service.cpp b/src/server/pegasus_manual_compact_service.cpp
index 22a40afcb..146db8f72 100644
--- a/src/server/pegasus_manual_compact_service.cpp
+++ b/src/server/pegasus_manual_compact_service.cpp
@@ -29,16 +29,27 @@
 #include "base/pegasus_const.h"
 #include "common/replication.codes.h"
 #include "pegasus_server_impl.h"
-#include "perf_counter/perf_counter.h"
 #include "runtime/api_layer1.h"
 #include "runtime/task/async_calls.h"
 #include "runtime/task/task_code.h"
+#include "utils/autoref_ptr.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
 #include "utils/string_conv.h"
+#include "utils/string_view.h"
 #include "utils/strings.h"
 #include "utils/time_utils.h"
 
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_manual_compact_queued_tasks,
+                          dsn::metric_unit::kTasks,
+                          "The number of current queued tasks of rocksdb manual compaction");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_manual_compact_running_tasks,
+                          dsn::metric_unit::kTasks,
+                          "The number of current running tasks of rocksdb manual compaction");
+
 namespace pegasus {
 namespace server {
 
@@ -58,17 +69,10 @@ pegasus_manual_compact_service::pegasus_manual_compact_service(pegasus_server_im
       _manual_compact_enqueue_time_ms(0),
       _manual_compact_start_running_time_ms(0),
       _manual_compact_last_finish_time_ms(0),
-      _manual_compact_last_time_used_ms(0)
+      _manual_compact_last_time_used_ms(0),
+      METRIC_VAR_INIT_replica(rdb_manual_compact_queued_tasks),
+      METRIC_VAR_INIT_replica(rdb_manual_compact_running_tasks)
 {
-    _pfc_manual_compact_enqueue_count.init_app_counter("app.pegasus",
-                                                       "manual.compact.enqueue.count",
-                                                       COUNTER_TYPE_NUMBER,
-                                                       "current manual compact in queue count");
-
-    _pfc_manual_compact_running_count.init_app_counter("app.pegasus",
-                                                       "manual.compact.running.count",
-                                                       COUNTER_TYPE_NUMBER,
-                                                       "current manual compact running count");
 }
 
 void pegasus_manual_compact_service::init_last_finish_time_ms(uint64_t last_finish_time_ms)
@@ -106,9 +110,9 @@ void pegasus_manual_compact_service::start_manual_compact_if_needed(
         rocksdb::CompactRangeOptions options;
         extract_manual_compact_opts(envs, compact_rule, options);
 
-        _pfc_manual_compact_enqueue_count->increment();
+        METRIC_VAR_INCREMENT(rdb_manual_compact_queued_tasks);
         dsn::tasking::enqueue(LPC_MANUAL_COMPACT, &_app->_tracker, [this, options]() {
-            _pfc_manual_compact_enqueue_count->decrement();
+            METRIC_VAR_DECREMENT(rdb_manual_compact_queued_tasks);
             manual_compact(options);
         });
     } else {
@@ -295,9 +299,8 @@ void pegasus_manual_compact_service::manual_compact(const rocksdb::CompactRangeO
     }
 
     // if current running count exceeds the limit, it would not to be started.
-    _pfc_manual_compact_running_count->increment();
-    if (_pfc_manual_compact_running_count->get_integer_value() > _max_concurrent_running_count) {
-        _pfc_manual_compact_running_count->decrement();
+    METRIC_VAR_AUTO_COUNT(rdb_manual_compact_running_tasks);
+    if (METRIC_VAR_VALUE(rdb_manual_compact_running_tasks) > _max_concurrent_running_count) {
         LOG_INFO_PREFIX("ignored compact because exceed max_concurrent_running_count({})",
                         _max_concurrent_running_count.load());
         _manual_compact_enqueue_time_ms.store(0);
@@ -307,8 +310,6 @@ void pegasus_manual_compact_service::manual_compact(const rocksdb::CompactRangeO
     uint64_t start = begin_manual_compact();
     uint64_t finish = _app->do_manual_compact(options);
     end_manual_compact(start, finish);
-
-    _pfc_manual_compact_running_count->decrement();
 }
 
 uint64_t pegasus_manual_compact_service::begin_manual_compact()
diff --git a/src/server/pegasus_manual_compact_service.h b/src/server/pegasus_manual_compact_service.h
index c47bbb5ec..d57343de2 100644
--- a/src/server/pegasus_manual_compact_service.h
+++ b/src/server/pegasus_manual_compact_service.h
@@ -25,8 +25,8 @@
 #include <string>
 
 #include "metadata_types.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica_base.h"
+#include "utils/metrics.h"
 
 namespace rocksdb {
 struct CompactRangeOptions;
@@ -97,8 +97,8 @@ private:
     std::atomic<uint64_t> _manual_compact_last_finish_time_ms;
     std::atomic<uint64_t> _manual_compact_last_time_used_ms;
 
-    ::dsn::perf_counter_wrapper _pfc_manual_compact_enqueue_count;
-    ::dsn::perf_counter_wrapper _pfc_manual_compact_running_count;
+    METRIC_VAR_DECLARE_gauge_int64(rdb_manual_compact_queued_tasks);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_manual_compact_running_tasks);
 };
 
 } // namespace server
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index bdc0ccd50..8b9d396a9 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -170,7 +170,7 @@ class error_code;
 #define METRIC_VAR_INIT_partition(name, ...) METRIC_VAR_INIT(name, partition, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_backup_policy(name, ...) METRIC_VAR_INIT(name, backup_policy, ##__VA_ARGS__)
 
-// Perform increment-related operations on metrics including gauge and counter.
+// Perform increment-related operations on gauges and counters.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
     do {                                                                                           \
         const auto v = (x);                                                                        \
@@ -179,9 +179,13 @@ class error_code;
         }                                                                                          \
     } while (0)
 
+// Perform increment() operations on gauges and counters.
 #define METRIC_VAR_INCREMENT(name) _##name->increment()
 
-// Perform set() operations on metrics including gauge and percentile.
+// Perform decrement() operations on gauges.
+#define METRIC_VAR_DECREMENT(name) _##name->decrement()
+
+// Perform set() operations on gauges and percentiles.
 //
 // There are 2 kinds of invocations of set() for a metric:
 // * set(val): set a single value for a metric, such as gauge, percentile;
@@ -189,7 +193,7 @@ class error_code;
 // such as percentile.
 #define METRIC_VAR_SET(name, ...) _##name->set(__VA_ARGS__)
 
-// Read the current measurement of the metric.
+// Read the current measurement of gauges and counters.
 #define METRIC_VAR_VALUE(name) _##name->value()
 
 // Convenient macro that is used to compute latency automatically, which is dedicated to percentile.
@@ -198,6 +202,10 @@ class error_code;
 
 #define METRIC_VAR_AUTO_LATENCY_DURATION_NS(name) __##name##_auto_latency.duration_ns()
 
+// Convenient macro that is used to increment/decrement gauge automatically in current scope.
+#define METRIC_VAR_AUTO_COUNT(name, ...)                                                           \
+    dsn::auto_count __##name##_auto_count(_##name, ##__VA_ARGS__)
+
 #define METRIC_DEFINE_INCREMENT_BY(name)                                                           \
     void increment_##name##_by(int64_t x) { METRIC_VAR_INCREMENT_BY(name, x); }
 
@@ -650,6 +658,7 @@ enum class metric_unit : size_t
     kWrites,
     kChanges,
     kOperations,
+    kTasks,
     kDisconnections,
     kServers,
     kInvalidUnit,
@@ -1433,22 +1442,22 @@ using floating_percentile_prototype =
 class auto_latency
 {
 public:
-    auto_latency(const percentile_ptr<int64_t> &percentile) : _percentile(percentile) {}
+    auto_latency(const percentile_ptr<int64_t> &p) : _percentile(p) {}
 
-    auto_latency(const percentile_ptr<int64_t> &percentile, std::function<void(uint64_t)> callback)
-        : _percentile(percentile), _callback(std::move(callback))
+    auto_latency(const percentile_ptr<int64_t> &p, std::function<void(uint64_t)> callback)
+        : _percentile(p), _callback(std::move(callback))
     {
     }
 
-    auto_latency(const percentile_ptr<int64_t> &percentile, uint64_t start_time_ns)
-        : _percentile(percentile), _chrono(start_time_ns)
+    auto_latency(const percentile_ptr<int64_t> &p, uint64_t start_time_ns)
+        : _percentile(p), _chrono(start_time_ns)
     {
     }
 
-    auto_latency(const percentile_ptr<int64_t> &percentile,
+    auto_latency(const percentile_ptr<int64_t> &p,
                  uint64_t start_time_ns,
                  std::function<void(uint64_t)> callback)
-        : _percentile(percentile), _chrono(start_time_ns), _callback(std::move(callback))
+        : _percentile(p), _chrono(start_time_ns), _callback(std::move(callback))
     {
     }
 
@@ -1473,6 +1482,34 @@ private:
     DISALLOW_COPY_AND_ASSIGN(auto_latency);
 };
 
+// Increment gauge and decrement it automatically at the end of the scope.
+class auto_count
+{
+public:
+    auto_count(const gauge_ptr<int64_t> &g) : _gauge(g) { _gauge->increment(); }
+
+    auto_count(const gauge_ptr<int64_t> &g, std::function<void()> callback)
+        : _gauge(g), _callback(std::move(callback))
+    {
+        _gauge->increment();
+    }
+
+    ~auto_count()
+    {
+        if (_callback) {
+            _callback();
+        }
+
+        _gauge->decrement();
+    }
+
+private:
+    gauge_ptr<int64_t> _gauge;
+    std::function<void()> _callback;
+
+    DISALLOW_COPY_AND_ASSIGN(auto_count);
+};
+
 } // namespace dsn
 
 // Since server_metric_entity() will be called in macros such as METRIC_VAR_INIT_server(), its
diff --git a/src/utils/test/metrics_test.cpp b/src/utils/test/metrics_test.cpp
index 9388a9a92..5197ec4b6 100644
--- a/src/utils/test/metrics_test.cpp
+++ b/src/utils/test/metrics_test.cpp
@@ -3096,6 +3096,8 @@ protected:
     void test_set_percentile(const std::vector<int64_t> &expected_samples);
     void test_set_percentile(size_t n, int64_t val);
 
+    void test_auto_count();
+
     const metric_entity_ptr _my_replica_metric_entity;
     METRIC_VAR_DECLARE_gauge_int64(test_replica_gauge_int64);
     METRIC_VAR_DECLARE_counter(test_replica_counter);
@@ -3134,6 +3136,19 @@ void MetricVarTest::test_set_percentile(size_t n, int64_t val)
     EXPECT_EQ(std::vector<int64_t>(n, val), METRIC_VAR_SAMPLES(test_replica_percentile_int64_ns));
 }
 
+void MetricVarTest::test_auto_count()
+{
+    ASSERT_EQ(0, METRIC_VAR_VALUE(test_replica_gauge_int64));
+
+    {
+        METRIC_VAR_AUTO_COUNT(test_replica_gauge_int64, [this]() {
+            ASSERT_EQ(1, METRIC_VAR_VALUE(test_replica_gauge_int64));
+        });
+    }
+
+    ASSERT_EQ(0, METRIC_VAR_VALUE(test_replica_gauge_int64));
+}
+
 #define TEST_METRIC_VAR_INCREMENT(name)                                                            \
     do {                                                                                           \
         ASSERT_EQ(0, METRIC_VAR_VALUE(name));                                                      \
@@ -3155,6 +3170,28 @@ TEST_F(MetricVarTest, IncrementGauge) { TEST_METRIC_VAR_INCREMENT(test_replica_g
 
 TEST_F(MetricVarTest, IncrementCounter) { TEST_METRIC_VAR_INCREMENT(test_replica_counter); }
 
+#define TEST_METRIC_VAR_DECREMENT(name)                                                            \
+    do {                                                                                           \
+        ASSERT_EQ(0, METRIC_VAR_VALUE(name));                                                      \
+                                                                                                   \
+        METRIC_VAR_INCREMENT_BY(name, 11);                                                         \
+        ASSERT_EQ(11, METRIC_VAR_VALUE(name));                                                     \
+                                                                                                   \
+        METRIC_VAR_DECREMENT(name);                                                                \
+        ASSERT_EQ(10, METRIC_VAR_VALUE(name));                                                     \
+                                                                                                   \
+        METRIC_VAR_DECREMENT(name);                                                                \
+        ASSERT_EQ(9, METRIC_VAR_VALUE(name));                                                      \
+                                                                                                   \
+        METRIC_VAR_INCREMENT(name);                                                                \
+        ASSERT_EQ(10, METRIC_VAR_VALUE(name));                                                     \
+                                                                                                   \
+        METRIC_VAR_DECREMENT(name);                                                                \
+        ASSERT_EQ(9, METRIC_VAR_VALUE(name));                                                      \
+    } while (0);
+
+TEST_F(MetricVarTest, DecrementGauge) { TEST_METRIC_VAR_DECREMENT(test_replica_gauge_int64); }
+
 TEST_F(MetricVarTest, SetGauge)
 {
     ASSERT_EQ(0, METRIC_VAR_VALUE(test_replica_gauge_int64));
@@ -3195,4 +3232,6 @@ TEST_F(MetricVarTest, AutoLatencyMilliSeconds) { TEST_METRIC_VAR_AUTO_LATENCY(ms
 
 TEST_F(MetricVarTest, AutoLatencySeconds) { TEST_METRIC_VAR_AUTO_LATENCY(s, 1000 * 1000 * 1000); }
 
+TEST_F(MetricVarTest, AutoCount) { ASSERT_NO_FATAL_FAILURE(test_auto_count()); }
+
 } // namespace dsn


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


[incubator-pegasus] 07/23: feat(new_metrics): migrate replica-level metrics for pegasus_event_listener (#1407)

Posted by wa...@apache.org.
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 da8bf31ed2cf668b97dae227b2dc220c253dc975
Author: Dan Wang <wa...@apache.org>
AuthorDate: Thu Mar 23 18:06:01 2023 +0800

    feat(new_metrics): migrate replica-level metrics for pegasus_event_listener (#1407)
    
    https://github.com/apache/incubator-pegasus/issues/1343
    
    Migrate replica-level metrics in pegasus_event_listener class to new framework, all of which
    are rocksdb-related, including the number of completed flushes/compactions, the size of
    flush output in bytes, the size of compaction input/output in bytes.
    
    Note that in old perf counters there are just 2 replica-level metrics for pegasus_event_listener
    while all of others are server-level. Migrated to new framework all of the metrics have become
    replica-level; once server-level metrics are needed, just aggregate on replica-level ones.
---
 src/server/pegasus_event_listener.cpp | 123 ++++++++++++++++------------------
 src/server/pegasus_event_listener.h   |  26 ++++---
 src/utils/metrics.h                   |   3 +
 3 files changed, 72 insertions(+), 80 deletions(-)

diff --git a/src/server/pegasus_event_listener.cpp b/src/server/pegasus_event_listener.cpp
index e5414d8b7..330f93af9 100644
--- a/src/server/pegasus_event_listener.cpp
+++ b/src/server/pegasus_event_listener.cpp
@@ -19,100 +19,91 @@
 
 #include "pegasus_event_listener.h"
 
-#include <fmt/core.h>
-#include <fmt/ostream.h>
 #include <rocksdb/compaction_job_stats.h>
 #include <rocksdb/table_properties.h>
-#include <iosfwd>
-#include <string>
 
-#include "common/gpid.h"
-#include "perf_counter/perf_counter.h"
+#include "utils/autoref_ptr.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
 
 namespace rocksdb {
 class DB;
 } // namespace rocksdb
 
+METRIC_DEFINE_counter(replica,
+                      rdb_flush_completed_count,
+                      dsn::metric_unit::kFlushes,
+                      "The number of completed rocksdb flushes");
+
+METRIC_DEFINE_counter(replica,
+                      rdb_flush_output_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The size of rocksdb flush output in bytes");
+
+METRIC_DEFINE_counter(replica,
+                      rdb_compaction_completed_count,
+                      dsn::metric_unit::kCompactions,
+                      "The number of completed rocksdb compactions");
+
+METRIC_DEFINE_counter(replica,
+                      rdb_compaction_input_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The size of rocksdb compaction input in bytes");
+
+METRIC_DEFINE_counter(replica,
+                      rdb_compaction_output_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The size of rocksdb compaction output in bytes");
+
+METRIC_DEFINE_counter(
+    replica,
+    rdb_changed_delayed_writes,
+    dsn::metric_unit::kWrites,
+    "The number of rocksdb delayed writes changed from another write stall condition");
+
+METRIC_DEFINE_counter(
+    replica,
+    rdb_changed_stopped_writes,
+    dsn::metric_unit::kWrites,
+    "The number of rocksdb stopped writes changed from another write stall condition");
+
 namespace pegasus {
 namespace server {
 
-pegasus_event_listener::pegasus_event_listener(replica_base *r) : replica_base(r)
+pegasus_event_listener::pegasus_event_listener(replica_base *r)
+    : replica_base(r),
+      METRIC_VAR_INIT_replica(rdb_flush_completed_count),
+      METRIC_VAR_INIT_replica(rdb_flush_output_bytes),
+      METRIC_VAR_INIT_replica(rdb_compaction_completed_count),
+      METRIC_VAR_INIT_replica(rdb_compaction_input_bytes),
+      METRIC_VAR_INIT_replica(rdb_compaction_output_bytes),
+      METRIC_VAR_INIT_replica(rdb_changed_delayed_writes),
+      METRIC_VAR_INIT_replica(rdb_changed_stopped_writes)
 {
-    _pfc_recent_flush_completed_count.init_app_counter("app.pegasus",
-                                                       "recent.flush.completed.count",
-                                                       COUNTER_TYPE_VOLATILE_NUMBER,
-                                                       "rocksdb recent flush completed count");
-    _pfc_recent_flush_output_bytes.init_app_counter("app.pegasus",
-                                                    "recent.flush.output.bytes",
-                                                    COUNTER_TYPE_VOLATILE_NUMBER,
-                                                    "rocksdb recent flush output bytes");
-    _pfc_recent_compaction_completed_count.init_app_counter(
-        "app.pegasus",
-        "recent.compaction.completed.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "rocksdb recent compaction completed count");
-    _pfc_recent_compaction_input_bytes.init_app_counter("app.pegasus",
-                                                        "recent.compaction.input.bytes",
-                                                        COUNTER_TYPE_VOLATILE_NUMBER,
-                                                        "rocksdb recent compaction input bytes");
-    _pfc_recent_compaction_output_bytes.init_app_counter("app.pegasus",
-                                                         "recent.compaction.output.bytes",
-                                                         COUNTER_TYPE_VOLATILE_NUMBER,
-                                                         "rocksdb recent compaction output bytes");
-    _pfc_recent_write_change_delayed_count.init_app_counter(
-        "app.pegasus",
-        "recent.write.change.delayed.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "rocksdb recent write change delayed count");
-    _pfc_recent_write_change_stopped_count.init_app_counter(
-        "app.pegasus",
-        "recent.write.change.stopped.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "rocksdb recent write change stopped count");
-
-    // replica-level perfcounter
-    std::string counter_str = fmt::format("recent_rdb_compaction_input_bytes@{}", r->get_gpid());
-    _pfc_recent_rdb_compaction_input_bytes.init_app_counter(
-        "app.pegasus",
-        counter_str.c_str(),
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "rocksdb recent compaction input bytes");
-
-    counter_str = fmt::format("recent_rdb_compaction_output_bytes@{}", r->get_gpid());
-    _pfc_recent_rdb_compaction_output_bytes.init_app_counter(
-        "app.pegasus",
-        counter_str.c_str(),
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "rocksdb recent compaction output bytes");
 }
 
-void pegasus_event_listener::OnFlushCompleted(rocksdb::DB *db,
-                                              const rocksdb::FlushJobInfo &flush_job_info)
+void pegasus_event_listener::OnFlushCompleted(rocksdb::DB *db, const rocksdb::FlushJobInfo &info)
 {
-    _pfc_recent_flush_completed_count->increment();
-    _pfc_recent_flush_output_bytes->add(flush_job_info.table_properties.data_size);
+    METRIC_VAR_INCREMENT(rdb_flush_completed_count);
+    METRIC_VAR_INCREMENT_BY(rdb_flush_output_bytes, info.table_properties.data_size);
 }
 
 void pegasus_event_listener::OnCompactionCompleted(rocksdb::DB *db,
-                                                   const rocksdb::CompactionJobInfo &ci)
+                                                   const rocksdb::CompactionJobInfo &info)
 {
-    _pfc_recent_compaction_completed_count->increment();
-    _pfc_recent_compaction_input_bytes->add(ci.stats.total_input_bytes);
-    _pfc_recent_compaction_output_bytes->add(ci.stats.total_output_bytes);
-
-    _pfc_recent_rdb_compaction_input_bytes->add(ci.stats.total_input_bytes);
-    _pfc_recent_rdb_compaction_output_bytes->add(ci.stats.total_output_bytes);
+    METRIC_VAR_INCREMENT(rdb_compaction_completed_count);
+    METRIC_VAR_INCREMENT_BY(rdb_compaction_input_bytes, info.stats.total_input_bytes);
+    METRIC_VAR_INCREMENT_BY(rdb_compaction_output_bytes, info.stats.total_output_bytes);
 }
 
 void pegasus_event_listener::OnStallConditionsChanged(const rocksdb::WriteStallInfo &info)
 {
     if (info.condition.cur == rocksdb::WriteStallCondition::kDelayed) {
         LOG_ERROR_PREFIX("rocksdb write delayed");
-        _pfc_recent_write_change_delayed_count->increment();
+        METRIC_VAR_INCREMENT(rdb_changed_delayed_writes);
     } else if (info.condition.cur == rocksdb::WriteStallCondition::kStopped) {
         LOG_ERROR_PREFIX("rocksdb write stopped");
-        _pfc_recent_write_change_stopped_count->increment();
+        METRIC_VAR_INCREMENT(rdb_changed_stopped_writes);
     }
 }
 
diff --git a/src/server/pegasus_event_listener.h b/src/server/pegasus_event_listener.h
index 5e6ab7e1a..900fabdc8 100644
--- a/src/server/pegasus_event_listener.h
+++ b/src/server/pegasus_event_listener.h
@@ -21,8 +21,8 @@
 
 #include <rocksdb/listener.h>
 
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica_base.h"
+#include "utils/metrics.h"
 
 namespace rocksdb {
 class DB;
@@ -37,24 +37,22 @@ public:
     explicit pegasus_event_listener(replica_base *r);
     ~pegasus_event_listener() override = default;
 
-    void OnFlushCompleted(rocksdb::DB *db, const rocksdb::FlushJobInfo &flush_job_info) override;
+    void OnFlushCompleted(rocksdb::DB *db, const rocksdb::FlushJobInfo &info) override;
 
-    void OnCompactionCompleted(rocksdb::DB *db, const rocksdb::CompactionJobInfo &ci) override;
+    void OnCompactionCompleted(rocksdb::DB *db, const rocksdb::CompactionJobInfo &info) override;
 
     void OnStallConditionsChanged(const rocksdb::WriteStallInfo &info) override;
 
 private:
-    ::dsn::perf_counter_wrapper _pfc_recent_flush_completed_count;
-    ::dsn::perf_counter_wrapper _pfc_recent_flush_output_bytes;
-    ::dsn::perf_counter_wrapper _pfc_recent_compaction_completed_count;
-    ::dsn::perf_counter_wrapper _pfc_recent_compaction_input_bytes;
-    ::dsn::perf_counter_wrapper _pfc_recent_compaction_output_bytes;
-    ::dsn::perf_counter_wrapper _pfc_recent_write_change_delayed_count;
-    ::dsn::perf_counter_wrapper _pfc_recent_write_change_stopped_count;
-
-    // replica-level perfcounter
-    ::dsn::perf_counter_wrapper _pfc_recent_rdb_compaction_input_bytes;
-    ::dsn::perf_counter_wrapper _pfc_recent_rdb_compaction_output_bytes;
+    METRIC_VAR_DECLARE_counter(rdb_flush_completed_count);
+    METRIC_VAR_DECLARE_counter(rdb_flush_output_bytes);
+
+    METRIC_VAR_DECLARE_counter(rdb_compaction_completed_count);
+    METRIC_VAR_DECLARE_counter(rdb_compaction_input_bytes);
+    METRIC_VAR_DECLARE_counter(rdb_compaction_output_bytes);
+
+    METRIC_VAR_DECLARE_counter(rdb_changed_delayed_writes);
+    METRIC_VAR_DECLARE_counter(rdb_changed_stopped_writes);
 };
 
 } // namespace server
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index 333534703..9087b78e5 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -620,6 +620,9 @@ enum class metric_unit : size_t
     kKeys,
     kFiles,
     kAmplification,
+    kFlushes,
+    kCompactions,
+    kWrites,
     kInvalidUnit,
 };
 


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


[incubator-pegasus] 15/23: feat(new_metrics): migrate server-level metrics for meta_service (#1437)

Posted by wa...@apache.org.
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 5325b76543f0730ff44dd7b1fe6954b9e1050a07
Author: Dan Wang <wa...@apache.org>
AuthorDate: Fri Apr 14 20:08:47 2023 +0800

    feat(new_metrics): migrate server-level metrics for meta_service (#1437)
    
    https://github.com/apache/incubator-pegasus/issues/1331
    
    Migrate metrics to new framework for meta_service, including the number
    of disconnections with replica servers, and the number of unalive and alive
    replica servers. All of these metrics are server-level, maintained in meta
    server.
    
    The old type in perf counters of the number of disconnections is volatile
    counter, which would be changed to non-volatile, while another 2 metrics
    would keep the type of gauge.
---
 src/meta/meta_service.cpp | 43 +++++++++++++++++++++++++++----------------
 src/meta/meta_service.h   |  8 ++++----
 src/utils/metrics.h       |  2 ++
 3 files changed, 33 insertions(+), 20 deletions(-)

diff --git a/src/meta/meta_service.cpp b/src/meta/meta_service.cpp
index 4b65d7b29..89864f753 100644
--- a/src/meta/meta_service.cpp
+++ b/src/meta/meta_service.cpp
@@ -55,7 +55,6 @@
 #include "meta_service.h"
 #include "meta_split_service.h"
 #include "partition_split_types.h"
-#include "perf_counter/perf_counter.h"
 #include "remote_cmd/remote_command.h"
 #include "runtime/ranger/ranger_resource_policy_manager.h"
 #include "runtime/rpc/rpc_holder.h"
@@ -68,8 +67,24 @@
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
 #include "utils/string_conv.h"
+#include "utils/string_view.h"
 #include "utils/strings.h"
 
+METRIC_DEFINE_counter(server,
+                      replica_server_disconnections,
+                      dsn::metric_unit::kDisconnections,
+                      "The number of disconnections with replica servers");
+
+METRIC_DEFINE_gauge_int64(server,
+                          unalive_replica_servers,
+                          dsn::metric_unit::kServers,
+                          "The number of unalive replica servers");
+
+METRIC_DEFINE_gauge_int64(server,
+                          alive_replica_servers,
+                          dsn::metric_unit::kServers,
+                          "The number of alive replica servers");
+
 namespace dsn {
 namespace dist {
 DSN_DECLARE_string(hosts_list);
@@ -142,7 +157,13 @@ DSN_DECLARE_string(cold_backup_root);
     } while (0)
 
 meta_service::meta_service()
-    : serverlet("meta_service"), _failure_detector(nullptr), _started(false), _recovering(false)
+    : serverlet("meta_service"),
+      _failure_detector(nullptr),
+      _started(false),
+      _recovering(false),
+      METRIC_VAR_INIT_server(replica_server_disconnections),
+      METRIC_VAR_INIT_server(unalive_replica_servers),
+      METRIC_VAR_INIT_server(alive_replica_servers)
 {
     _opts.initialize();
     _meta_opts.initialize();
@@ -158,16 +179,6 @@ meta_service::meta_service()
         }
     }
 
-    _recent_disconnect_count.init_app_counter(
-        "eon.meta_service",
-        "recent_disconnect_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "replica server disconnect count in the recent period");
-    _unalive_nodes_count.init_app_counter(
-        "eon.meta_service", "unalive_nodes", COUNTER_TYPE_NUMBER, "current count of unalive nodes");
-    _alive_nodes_count.init_app_counter(
-        "eon.meta_service", "alive_nodes", COUNTER_TYPE_NUMBER, "current count of alive nodes");
-
     _meta_op_status.store(meta_op_status::FREE);
 }
 
@@ -242,9 +253,9 @@ void meta_service::set_node_state(const std::vector<rpc_address> &nodes, bool is
         }
     }
 
-    _recent_disconnect_count->add(is_alive ? 0 : nodes.size());
-    _unalive_nodes_count->set(_dead_set.size());
-    _alive_nodes_count->set(_alive_set.size());
+    METRIC_VAR_INCREMENT_BY(replica_server_disconnections, is_alive ? 0 : nodes.size());
+    METRIC_VAR_SET(unalive_replica_servers, _dead_set.size());
+    METRIC_VAR_SET(alive_replica_servers, _alive_set.size());
 
     if (!_started) {
         return;
@@ -327,7 +338,7 @@ void meta_service::start_service()
             _alive_set.insert(kv.first);
     }
 
-    _alive_nodes_count->set(_alive_set.size());
+    METRIC_VAR_SET(alive_replica_servers, _alive_set.size());
 
     for (const dsn::rpc_address &node : _alive_set) {
         // sync alive set and the failure_detector
diff --git a/src/meta/meta_service.h b/src/meta/meta_service.h
index 58f5a902a..caa01d79a 100644
--- a/src/meta/meta_service.h
+++ b/src/meta/meta_service.h
@@ -55,7 +55,6 @@
 #include "meta_options.h"
 #include "meta_rpc_types.h"
 #include "meta_server_failure_detector.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "runtime/api_layer1.h"
 #include "runtime/rpc/network.h"
 #include "runtime/rpc/rpc_address.h"
@@ -70,6 +69,7 @@
 #include "utils/enum_helper.h"
 #include "utils/error_code.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/threadpool_code.h"
 #include "utils/zlocks.h"
 
@@ -383,9 +383,9 @@ private:
 
     std::string _cluster_root;
 
-    perf_counter_wrapper _recent_disconnect_count;
-    perf_counter_wrapper _unalive_nodes_count;
-    perf_counter_wrapper _alive_nodes_count;
+    METRIC_VAR_DECLARE_counter(replica_server_disconnections);
+    METRIC_VAR_DECLARE_gauge_int64(unalive_replica_servers);
+    METRIC_VAR_DECLARE_gauge_int64(alive_replica_servers);
 
     dsn::task_tracker _tracker;
 
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index cb6b7cb48..377bb58d7 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -649,6 +649,8 @@ enum class metric_unit : size_t
     kWrites,
     kChanges,
     kOperations,
+    kDisconnections,
+    kServers,
     kInvalidUnit,
 };
 


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


[incubator-pegasus] 20/23: feat(new_metrics): migrate metrics for replica_stub (part 2) (#1459)

Posted by wa...@apache.org.
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 0d46760c60947c5707bb21e0be4d5da13cc19c55
Author: Dan Wang <wa...@apache.org>
AuthorDate: Wed Apr 26 11:03:06 2023 +0800

    feat(new_metrics): migrate metrics for replica_stub (part 2) (#1459)
    
    https://github.com/apache/incubator-pegasus/issues/1454
    
    This is the 2nd part of migrating metrics of `replica_stub` to new framework,
    all of which are learn-related.
    
    During this migration, there are 3 metrics still keeping server-level, including
    the number of learning replicas, the max duration and the max size of files that
    are copied from learnee among all learning replicas.
    
    Another 11 metrics are changed from server-level to replica-level, since they
    should be observed for each replica. All of them are observed from the view
    of learners, namely the potential secondary replica. The learnee is the primary
    replica. These metrics include: the number of learns launched by learner, the
    number of learn rounds launched by learner (during a learn there might be
    multiple rounds, the number of files that are copied from learnee, the size of
    files that are copied from learnee, the size of data that are copied from learnee's
    buffer, the number of learn responses of `LT_CACHE`, `LT_APP` and `LT_LOG`
    type decided by learner with each learn response related to an `RPC_LEARN`
    request, the number of times learner resets its local state (since its local state
    is newer than learnee's) with each reset related to an learn response of an
    `RPC_LEARN` request, the number of failed and successful learns launched by
    learner.
---
 src/replica/replica.cpp       | 75 ++++++++++++++++++++++++++++++++-
 src/replica/replica.h         | 12 ++++++
 src/replica/replica_learn.cpp | 39 ++++++++++-------
 src/replica/replica_stub.cpp  | 98 ++++++++++---------------------------------
 src/replica/replica_stub.h    | 17 ++------
 src/utils/metrics.h           |  4 ++
 6 files changed, 141 insertions(+), 104 deletions(-)

diff --git a/src/replica/replica.cpp b/src/replica/replica.cpp
index 392c05300..349bae53d 100644
--- a/src/replica/replica.cpp
+++ b/src/replica/replica.cpp
@@ -127,6 +127,68 @@ METRIC_DEFINE_counter(replica,
                       dsn::metric_unit::kRequests,
                       "The number of rejected non-idempotent write requests by duplication");
 
+METRIC_DEFINE_counter(
+    replica,
+    learn_count,
+    dsn::metric_unit::kLearns,
+    "The number of learns launched by learner (i.e. potential secondary replica)");
+
+METRIC_DEFINE_counter(replica,
+                      learn_rounds,
+                      dsn::metric_unit::kRounds,
+                      "The number of learn rounds launched by learner (during a learn there might"
+                      "be multiple rounds)");
+
+METRIC_DEFINE_counter(replica,
+                      learn_copy_files,
+                      dsn::metric_unit::kFiles,
+                      "The number of files that are copied from learnee (i.e. primary replica)");
+
+METRIC_DEFINE_counter(replica,
+                      learn_copy_file_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The size of file that are copied from learnee");
+
+METRIC_DEFINE_counter(replica,
+                      learn_copy_buffer_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The size of data that are copied from learnee's buffer");
+
+METRIC_DEFINE_counter(replica,
+                      learn_lt_cache_responses,
+                      dsn::metric_unit::kResponses,
+                      "The number of learn responses of LT_CACHE type decided by learner, with "
+                      "each learn response related to an RPC_LEARN request");
+
+METRIC_DEFINE_counter(replica,
+                      learn_lt_app_responses,
+                      dsn::metric_unit::kResponses,
+                      "The number of learn responses of LT_APP type decided by learner, with each "
+                      "learn response related to an RPC_LEARN request");
+
+METRIC_DEFINE_counter(replica,
+                      learn_lt_log_responses,
+                      dsn::metric_unit::kResponses,
+                      "The number of learn responses of LT_LOG type decided by learner, with each "
+                      "learn response related to an RPC_LEARN request");
+
+METRIC_DEFINE_counter(replica,
+                      learn_resets,
+                      dsn::metric_unit::kResets,
+                      "The number of times learner resets its local state (since its local state "
+                      "is newer than learnee's), with each reset related to an learn response of "
+                      "an RPC_LEARN request");
+
+METRIC_DEFINE_counter(replica,
+                      learn_failed_count,
+                      dsn::metric_unit::kLearns,
+                      "The number of failed learns launched by learner");
+
+METRIC_DEFINE_counter(replica,
+                      learn_successful_count,
+                      dsn::metric_unit::kLearns,
+                      "The number of successful learns launched by learner");
+
 namespace dsn {
 namespace replication {
 
@@ -191,7 +253,18 @@ replica::replica(replica_stub *stub,
       METRIC_VAR_INIT_replica(splitting_rejected_write_requests),
       METRIC_VAR_INIT_replica(splitting_rejected_read_requests),
       METRIC_VAR_INIT_replica(bulk_load_ingestion_rejected_write_requests),
-      METRIC_VAR_INIT_replica(dup_rejected_non_idempotent_write_requests)
+      METRIC_VAR_INIT_replica(dup_rejected_non_idempotent_write_requests),
+      METRIC_VAR_INIT_replica(learn_count),
+      METRIC_VAR_INIT_replica(learn_rounds),
+      METRIC_VAR_INIT_replica(learn_copy_files),
+      METRIC_VAR_INIT_replica(learn_copy_file_bytes),
+      METRIC_VAR_INIT_replica(learn_copy_buffer_bytes),
+      METRIC_VAR_INIT_replica(learn_lt_cache_responses),
+      METRIC_VAR_INIT_replica(learn_lt_app_responses),
+      METRIC_VAR_INIT_replica(learn_lt_log_responses),
+      METRIC_VAR_INIT_replica(learn_resets),
+      METRIC_VAR_INIT_replica(learn_failed_count),
+      METRIC_VAR_INIT_replica(learn_successful_count)
 {
     CHECK(!_app_info.app_type.empty(), "");
     CHECK_NOTNULL(stub, "");
diff --git a/src/replica/replica.h b/src/replica/replica.h
index ab9405895..25494fd2c 100644
--- a/src/replica/replica.h
+++ b/src/replica/replica.h
@@ -664,6 +664,18 @@ private:
     METRIC_VAR_DECLARE_counter(dup_rejected_non_idempotent_write_requests);
     std::vector<perf_counter *> _counters_table_level_latency;
 
+    METRIC_VAR_DECLARE_counter(learn_count);
+    METRIC_VAR_DECLARE_counter(learn_rounds);
+    METRIC_VAR_DECLARE_counter(learn_copy_files);
+    METRIC_VAR_DECLARE_counter(learn_copy_file_bytes);
+    METRIC_VAR_DECLARE_counter(learn_copy_buffer_bytes);
+    METRIC_VAR_DECLARE_counter(learn_lt_cache_responses);
+    METRIC_VAR_DECLARE_counter(learn_lt_app_responses);
+    METRIC_VAR_DECLARE_counter(learn_lt_log_responses);
+    METRIC_VAR_DECLARE_counter(learn_resets);
+    METRIC_VAR_DECLARE_counter(learn_failed_count);
+    METRIC_VAR_DECLARE_counter(learn_successful_count);
+
     dsn::task_tracker _tracker;
     // the thread access checker
     dsn::thread_access_checker _checker;
diff --git a/src/replica/replica_learn.cpp b/src/replica/replica_learn.cpp
index 589e3e28d..b123fbe83 100644
--- a/src/replica/replica_learn.cpp
+++ b/src/replica/replica_learn.cpp
@@ -57,8 +57,6 @@
 #include "mutation.h"
 #include "mutation_log.h"
 #include "nfs/nfs_node.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica.h"
 #include "replica/duplication/replica_duplicator_manager.h"
 #include "replica/prepare_list.h"
@@ -79,8 +77,21 @@
 #include "utils/filesystem.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/thread_access_checker.h"
 
+METRIC_DECLARE_counter(learn_count);
+METRIC_DECLARE_counter(learn_rounds);
+METRIC_DECLARE_counter(learn_copy_files);
+METRIC_DECLARE_counter(learn_copy_file_bytes);
+METRIC_DECLARE_counter(learn_copy_buffer_bytes);
+METRIC_DECLARE_counter(learn_lt_cache_responses);
+METRIC_DECLARE_counter(learn_lt_app_responses);
+METRIC_DECLARE_counter(learn_lt_log_responses);
+METRIC_DECLARE_counter(learn_resets);
+METRIC_DECLARE_counter(learn_failed_count);
+METRIC_DECLARE_counter(learn_successful_count);
+
 namespace dsn {
 namespace replication {
 
@@ -136,7 +147,7 @@ void replica::init_learn(uint64_t signature)
             return;
         }
 
-        _stub->_counter_replicas_learning_recent_start_count->increment();
+        METRIC_VAR_INCREMENT(learn_count);
 
         _potential_secondary_states.learning_version = signature;
         _potential_secondary_states.learning_start_ts_ns = dsn_now_ns();
@@ -176,7 +187,7 @@ void replica::init_learn(uint64_t signature)
 
                     // missed ones need to be loaded via private logs
                     else {
-                        _stub->_counter_replicas_learning_recent_round_start_count->increment();
+                        METRIC_VAR_INCREMENT(learn_rounds);
                         _potential_secondary_states.learning_round_is_running = true;
                         _potential_secondary_states.catchup_with_private_log_task =
                             tasking::create_task(LPC_CATCHUP_WITH_PRIVATE_LOGS,
@@ -230,7 +241,7 @@ void replica::init_learn(uint64_t signature)
         return;
     }
 
-    _stub->_counter_replicas_learning_recent_round_start_count->increment();
+    METRIC_VAR_INCREMENT(learn_rounds);
     _potential_secondary_states.learning_round_is_running = true;
 
     learn_request request;
@@ -601,7 +612,7 @@ void replica::on_learn_reply(error_code err, learn_request &&req, learn_response
         enum_to_string(_potential_secondary_states.learning_status));
 
     _potential_secondary_states.learning_copy_buffer_size += resp.state.meta.length();
-    _stub->_counter_replicas_learning_recent_copy_buffer_size->add(resp.state.meta.length());
+    METRIC_VAR_INCREMENT_BY(learn_copy_buffer_bytes, resp.state.meta.length());
 
     if (resp.err != ERR_OK) {
         if (resp.err == ERR_INACTIVE_STATE || resp.err == ERR_INCONSISTENT_STATE) {
@@ -649,7 +660,7 @@ void replica::on_learn_reply(error_code err, learn_request &&req, learn_response
                            _app->last_committed_decree(),
                            resp.last_committed_decree);
 
-        _stub->_counter_replicas_learning_recent_learn_reset_count->increment();
+        METRIC_VAR_INCREMENT(learn_resets);
 
         // close app
         auto err = _app->close(true);
@@ -742,13 +753,13 @@ void replica::on_learn_reply(error_code err, learn_request &&req, learn_response
 
     switch (resp.type) {
     case learn_type::LT_CACHE:
-        _stub->_counter_replicas_learning_recent_learn_cache_count->increment();
+        METRIC_VAR_INCREMENT(learn_lt_cache_responses);
         break;
     case learn_type::LT_APP:
-        _stub->_counter_replicas_learning_recent_learn_app_count->increment();
+        METRIC_VAR_INCREMENT(learn_lt_app_responses);
         break;
     case learn_type::LT_LOG:
-        _stub->_counter_replicas_learning_recent_learn_log_count->increment();
+        METRIC_VAR_INCREMENT(learn_lt_log_responses);
         break;
     default:
         // do nothing
@@ -1043,8 +1054,8 @@ void replica::on_copy_remote_state_completed(error_code err,
     if (err == ERR_OK) {
         _potential_secondary_states.learning_copy_file_count += resp.state.files.size();
         _potential_secondary_states.learning_copy_file_size += size;
-        _stub->_counter_replicas_learning_recent_copy_file_count->add(resp.state.files.size());
-        _stub->_counter_replicas_learning_recent_copy_file_size->add(size);
+        METRIC_VAR_INCREMENT_BY(learn_copy_files, resp.state.files.size());
+        METRIC_VAR_INCREMENT_BY(learn_copy_file_bytes, size);
     }
 
     if (err != ERR_OK) {
@@ -1241,7 +1252,7 @@ void replica::handle_learning_error(error_code err, bool is_local_error)
         _data_corrupted = true;
     }
 
-    _stub->_counter_replicas_learning_recent_learn_fail_count->increment();
+    METRIC_VAR_INCREMENT(learn_failed_count);
 
     update_local_configuration_with_no_ballot_change(
         is_local_error ? partition_status::PS_ERROR : partition_status::PS_INACTIVE);
@@ -1405,7 +1416,7 @@ void replica::on_learn_completion_notification_reply(error_code err,
             handle_learning_error(resp.err, false);
         }
     } else {
-        _stub->_counter_replicas_learning_recent_learn_succ_count->increment();
+        METRIC_VAR_INCREMENT(learn_successful_count);
     }
 }
 
diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp
index 292eda470..35c0b742d 100644
--- a/src/replica/replica_stub.cpp
+++ b/src/replica/replica_stub.cpp
@@ -113,6 +113,22 @@ METRIC_DEFINE_gauge_int64(server,
                           dsn::metric_unit::kReplicas,
                           "The number of closing replicas");
 
+METRIC_DEFINE_gauge_int64(server,
+                          learning_replicas,
+                          dsn::metric_unit::kReplicas,
+                          "The number of learning replicas");
+
+METRIC_DEFINE_gauge_int64(server,
+                          learning_replicas_max_duration_ms,
+                          dsn::metric_unit::kMilliSeconds,
+                          "The max duration among all learning replicas");
+
+METRIC_DEFINE_gauge_int64(
+    server,
+    learning_replicas_max_copy_file_bytes,
+    dsn::metric_unit::kBytes,
+    "The max size of files that are copied from learnee among all learning replicas");
+
 namespace dsn {
 namespace replication {
 
@@ -218,7 +234,10 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/,
       _is_running(false),
       METRIC_VAR_INIT_server(total_replicas),
       METRIC_VAR_INIT_server(opening_replicas),
-      METRIC_VAR_INIT_server(closing_replicas)
+      METRIC_VAR_INIT_server(closing_replicas),
+      METRIC_VAR_INIT_server(learning_replicas),
+      METRIC_VAR_INIT_server(learning_replicas_max_duration_ms),
+      METRIC_VAR_INIT_server(learning_replicas_max_copy_file_bytes)
 {
 #ifdef DSN_ENABLE_GPERF
     _is_releasing_memory = false;
@@ -236,77 +255,6 @@ replica_stub::~replica_stub(void) { close(); }
 
 void replica_stub::install_perf_counters()
 {
-    _counter_replicas_learning_count.init_app_counter("eon.replica_stub",
-                                                      "replicas.learning.count",
-                                                      COUNTER_TYPE_NUMBER,
-                                                      "current learning count");
-    _counter_replicas_learning_max_duration_time_ms.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.max.duration.time(ms)",
-        COUNTER_TYPE_NUMBER,
-        "current learning max duration time(ms)");
-    _counter_replicas_learning_max_copy_file_size.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.max.copy.file.size",
-        COUNTER_TYPE_NUMBER,
-        "current learning max copy file size");
-    _counter_replicas_learning_recent_start_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.start.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "current learning start count in the recent period");
-    _counter_replicas_learning_recent_round_start_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.round.start.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning round start count in the recent period");
-    _counter_replicas_learning_recent_copy_file_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.copy.file.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning copy file count in the recent period");
-    _counter_replicas_learning_recent_copy_file_size.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.copy.file.size",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning copy file size in the recent period");
-    _counter_replicas_learning_recent_copy_buffer_size.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.copy.buffer.size",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning copy buffer size in the recent period");
-    _counter_replicas_learning_recent_learn_cache_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.learn.cache.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning LT_CACHE count in the recent period");
-    _counter_replicas_learning_recent_learn_app_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.learn.app.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning LT_APP count in the recent period");
-    _counter_replicas_learning_recent_learn_log_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.learn.log.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning LT_LOG count in the recent period");
-    _counter_replicas_learning_recent_learn_reset_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.learn.reset.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning reset count in the recent period"
-        "for the reason of resp.last_committed_decree < _app->last_committed_decree()");
-    _counter_replicas_learning_recent_learn_fail_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.learn.fail.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning fail count in the recent period");
-    _counter_replicas_learning_recent_learn_succ_count.init_app_counter(
-        "eon.replica_stub",
-        "replicas.learning.recent.learn.succ.count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "learning succeed count in the recent period");
-
     _counter_replicas_recent_prepare_fail_count.init_app_counter(
         "eon.replica_stub",
         "replicas.recent.prepare.fail.count",
@@ -1997,9 +1945,9 @@ void replica_stub::on_gc()
         }
     }
 
-    _counter_replicas_learning_count->set(learning_count);
-    _counter_replicas_learning_max_duration_time_ms->set(learning_max_duration_time_ms);
-    _counter_replicas_learning_max_copy_file_size->set(learning_max_copy_file_size);
+    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);
diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h
index 21e68c843..adaef7220 100644
--- a/src/replica/replica_stub.h
+++ b/src/replica/replica_stub.h
@@ -509,20 +509,9 @@ private:
     METRIC_VAR_DECLARE_gauge_int64(opening_replicas);
     METRIC_VAR_DECLARE_gauge_int64(closing_replicas);
 
-    perf_counter_wrapper _counter_replicas_learning_count;
-    perf_counter_wrapper _counter_replicas_learning_max_duration_time_ms;
-    perf_counter_wrapper _counter_replicas_learning_max_copy_file_size;
-    perf_counter_wrapper _counter_replicas_learning_recent_start_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_round_start_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_copy_file_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_copy_file_size;
-    perf_counter_wrapper _counter_replicas_learning_recent_copy_buffer_size;
-    perf_counter_wrapper _counter_replicas_learning_recent_learn_cache_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_learn_app_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_learn_log_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_learn_reset_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_learn_fail_count;
-    perf_counter_wrapper _counter_replicas_learning_recent_learn_succ_count;
+    METRIC_VAR_DECLARE_gauge_int64(learning_replicas);
+    METRIC_VAR_DECLARE_gauge_int64(learning_replicas_max_duration_ms);
+    METRIC_VAR_DECLARE_gauge_int64(learning_replicas_max_copy_file_bytes);
 
     perf_counter_wrapper _counter_replicas_recent_prepare_fail_count;
     perf_counter_wrapper _counter_replicas_recent_replica_move_error_count;
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index f9ab9c2dc..b5e31c050 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -656,6 +656,7 @@ enum class metric_unit : size_t
     kReplicas,
     kServers,
     kRequests,
+    kResponses,
     kSeeks,
     kPointLookups,
     kValues,
@@ -669,6 +670,9 @@ enum class metric_unit : size_t
     kOperations,
     kTasks,
     kDisconnections,
+    kLearns,
+    kRounds,
+    kResets,
     kInvalidUnit,
 };
 


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


[incubator-pegasus] 04/23: feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 2) (#1386)

Posted by wa...@apache.org.
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 ecce3000c5f923f306717c5c4f3f14eac338e5de
Author: Dan Wang <wa...@apache.org>
AuthorDate: Fri Mar 10 17:05:59 2023 +0800

    feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 2) (#1386)
    
    This PR is to migrate replica-level metrics of pegasus_server_impl to new framework,
    2nd part, for #1333.
    
    This PR focuses on migrating all rocksdb-related metrics for each replica, including
    total number and size of sst files, estimated number of keys, memory usage and hit
    rate, write/read amplification, negatives/positives of bloom filters.
---
 src/server/pegasus_server_impl.cpp      | 150 ++++++-----------
 src/server/pegasus_server_impl.h        |  48 +++---
 src/server/pegasus_server_impl_init.cpp | 275 ++++++++++++++++++--------------
 src/utils/metrics.h                     |  12 +-
 4 files changed, 238 insertions(+), 247 deletions(-)

diff --git a/src/server/pegasus_server_impl.cpp b/src/server/pegasus_server_impl.cpp
index ab6c4dc1c..7d0dad595 100644
--- a/src/server/pegasus_server_impl.cpp
+++ b/src/server/pegasus_server_impl.cpp
@@ -360,6 +360,15 @@ void pegasus_server_impl::log_expired_data(const char *op,
         }                                                                                          \
     } while (0)
 
+#define CHECK_READ_THROTTLING()                                                                    \
+    do {                                                                                           \
+        if (dsn_unlikely(!_read_size_throttling_controller->available())) {                        \
+            rpc.error() = dsn::ERR_BUSY;                                                           \
+            METRIC_VAR_INCREMENT(throttling_rejected_read_requests);                               \
+            return;                                                                                \
+        }                                                                                          \
+    } while (0)
+
 void pegasus_server_impl::on_get(get_rpc rpc)
 {
     CHECK_TRUE(_is_open);
@@ -376,11 +385,7 @@ void pegasus_server_impl::on_get(get_rpc rpc)
         return;
     }
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     METRIC_VAR_AUTO_LATENCY(get_latency_ns);
 
@@ -453,11 +458,7 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
     resp.partition_index = _gpid.get_partition_index();
     resp.server = _primary_address;
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     METRIC_VAR_AUTO_LATENCY(multi_get_latency_ns);
 
@@ -866,11 +867,7 @@ void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
     response.partition_index = _gpid.get_partition_index();
     response.server = _primary_address;
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     METRIC_VAR_AUTO_LATENCY(batch_get_latency_ns);
 
@@ -982,11 +979,7 @@ void pegasus_server_impl::on_sortkey_count(sortkey_count_rpc rpc)
     resp.partition_index = _gpid.get_partition_index();
     resp.server = _primary_address;
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     METRIC_VAR_AUTO_LATENCY(scan_latency_ns);
 
@@ -1059,11 +1052,7 @@ void pegasus_server_impl::on_ttl(ttl_rpc rpc)
     resp.partition_index = _gpid.get_partition_index();
     resp.server = _primary_address;
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     rocksdb::Slice skey(key.data(), key.length());
     std::string value;
@@ -1123,11 +1112,7 @@ void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
     resp.partition_index = _gpid.get_partition_index();
     resp.server = _primary_address;
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     METRIC_VAR_AUTO_LATENCY(scan_latency_ns);
 
@@ -1374,11 +1359,7 @@ void pegasus_server_impl::on_scan(scan_rpc rpc)
     resp.partition_index = _gpid.get_partition_index();
     resp.server = _primary_address;
 
-    if (!_read_size_throttling_controller->available()) {
-        rpc.error() = dsn::ERR_BUSY;
-        _counter_recent_read_throttling_reject_count->increment();
-        return;
-    }
+    CHECK_READ_THROTTLING();
 
     METRIC_VAR_AUTO_LATENCY(scan_latency_ns);
 
@@ -1877,13 +1858,13 @@ void pegasus_server_impl::cancel_background_work(bool wait)
             LOG_ERROR_PREFIX("rmdir {} failed when stop app", data_dir());
             return ::dsn::ERR_FILE_OPERATION_FAILED;
         }
-        _pfc_rdb_sst_count->set(0);
-        _pfc_rdb_sst_size->set(0);
-        _pfc_rdb_block_cache_hit_count->set(0);
-        _pfc_rdb_block_cache_total_count->set(0);
+        METRIC_VAR_SET(rdb_total_sst_files, 0);
+        METRIC_VAR_SET(rdb_total_sst_size_mb, 0);
+        METRIC_VAR_SET(rdb_index_and_filter_blocks_mem_usage_bytes, 0);
+        METRIC_VAR_SET(rdb_memtable_mem_usage_bytes, 0);
+        METRIC_VAR_SET(rdb_block_cache_hit_count, 0);
+        METRIC_VAR_SET(rdb_block_cache_total_count, 0);
         _pfc_rdb_block_cache_mem_usage->set(0);
-        _pfc_rdb_index_and_filter_blocks_mem_usage->set(0);
-        _pfc_rdb_memtable_mem_usage->set(0);
     }
 
     LOG_INFO_PREFIX("close app succeed, clear_state = {}", clear_state ? "true" : "false");
@@ -2447,12 +2428,16 @@ range_iteration_state pegasus_server_impl::append_key_value_for_multi_get(
     return range_iteration_state::kNormal;
 }
 
+#define GET_TICKER_COUNT_AND_SET_METRIC(ticker_name, metric_name)                                  \
+    do {                                                                                           \
+        METRIC_VAR_SET(metric_name, _statistics->getTickerCount(rocksdb::ticker_name));            \
+    } while (0)
+
 void pegasus_server_impl::update_replica_rocksdb_statistics()
 {
     std::string str_val;
     uint64_t val = 0;
 
-    // Update _pfc_rdb_sst_count
     for (int i = 0; i < _data_cf_opts.num_levels; ++i) {
         int cur_level_count = 0;
         if (_db->GetProperty(rocksdb::DB::Properties::kNumFilesAtLevelPrefix + std::to_string(i),
@@ -2461,49 +2446,38 @@ void pegasus_server_impl::update_replica_rocksdb_statistics()
             val += cur_level_count;
         }
     }
-    _pfc_rdb_sst_count->set(val);
-    LOG_DEBUG_PREFIX("_pfc_rdb_sst_count: {}", val);
+    METRIC_VAR_SET(rdb_total_sst_files, val);
 
-    // Update _pfc_rdb_sst_size
     if (_db->GetProperty(_data_cf, rocksdb::DB::Properties::kTotalSstFilesSize, &str_val) &&
         dsn::buf2uint64(str_val, val)) {
         static uint64_t bytes_per_mb = 1U << 20U;
-        _pfc_rdb_sst_size->set(val / bytes_per_mb);
-        LOG_DEBUG_PREFIX("_pfc_rdb_sst_size: {} bytes", val);
+        METRIC_VAR_SET(rdb_total_sst_size_mb, val / bytes_per_mb);
     }
 
-    // Update _pfc_rdb_write_amplification
     std::map<std::string, std::string> props;
     if (_db->GetMapProperty(_data_cf, "rocksdb.cfstats", &props)) {
         auto write_amplification_iter = props.find("compaction.Sum.WriteAmp");
         auto write_amplification = write_amplification_iter == props.end()
                                        ? 1
                                        : std::stod(write_amplification_iter->second);
-        _pfc_rdb_write_amplification->set(write_amplification);
-        LOG_DEBUG_PREFIX("_pfc_rdb_write_amplification: {}", write_amplification);
+        METRIC_VAR_SET(rdb_write_amplification, write_amplification);
     }
 
-    // Update _pfc_rdb_index_and_filter_blocks_mem_usage
     if (_db->GetProperty(_data_cf, rocksdb::DB::Properties::kEstimateTableReadersMem, &str_val) &&
         dsn::buf2uint64(str_val, val)) {
-        _pfc_rdb_index_and_filter_blocks_mem_usage->set(val);
-        LOG_DEBUG_PREFIX("_pfc_rdb_index_and_filter_blocks_mem_usage: {} bytes", val);
+        METRIC_VAR_SET(rdb_index_and_filter_blocks_mem_usage_bytes, val);
     }
 
-    // Update _pfc_rdb_memtable_mem_usage
     if (_db->GetProperty(_data_cf, rocksdb::DB::Properties::kCurSizeAllMemTables, &str_val) &&
         dsn::buf2uint64(str_val, val)) {
-        _pfc_rdb_memtable_mem_usage->set(val);
-        LOG_DEBUG_PREFIX("_pfc_rdb_memtable_mem_usage: {} bytes", val);
+        METRIC_VAR_SET(rdb_memtable_mem_usage_bytes, val);
     }
 
-    // Update _pfc_rdb_estimate_num_keys
     // NOTE: for the same n kv pairs, kEstimateNumKeys will be counted n times, you need compaction
     // to remove duplicate
     if (_db->GetProperty(_data_cf, rocksdb::DB::Properties::kEstimateNumKeys, &str_val) &&
         dsn::buf2uint64(str_val, val)) {
-        _pfc_rdb_estimate_num_keys->set(val);
-        LOG_DEBUG_PREFIX("_pfc_rdb_estimate_num_keys: {}", val);
+        METRIC_VAR_SET(rdb_estimated_keys, val);
     }
 
     // the follow stats is related to `read`, so only primary need update it,ignore
@@ -2512,7 +2486,6 @@ void pegasus_server_impl::update_replica_rocksdb_statistics()
         return;
     }
 
-    // Update _pfc_rdb_read_amplification
     if (FLAGS_read_amp_bytes_per_bit > 0) {
         auto estimate_useful_bytes =
             _statistics->getTickerCount(rocksdb::READ_AMP_ESTIMATE_USEFUL_BYTES);
@@ -2520,68 +2493,41 @@ void pegasus_server_impl::update_replica_rocksdb_statistics()
             auto read_amplification =
                 _statistics->getTickerCount(rocksdb::READ_AMP_TOTAL_READ_BYTES) /
                 estimate_useful_bytes;
-            _pfc_rdb_read_amplification->set(read_amplification);
-            LOG_DEBUG_PREFIX("_pfc_rdb_read_amplification: {}", read_amplification);
+            METRIC_VAR_SET(rdb_read_amplification, read_amplification);
         }
     }
 
-    // Update _pfc_rdb_bf_seek_negatives
-    auto bf_seek_negatives = _statistics->getTickerCount(rocksdb::BLOOM_FILTER_PREFIX_USEFUL);
-    _pfc_rdb_bf_seek_negatives->set(bf_seek_negatives);
-    LOG_DEBUG_PREFIX("_pfc_rdb_bf_seek_negatives: {}", bf_seek_negatives);
+    GET_TICKER_COUNT_AND_SET_METRIC(BLOOM_FILTER_PREFIX_USEFUL, rdb_bloom_filter_seek_negatives);
 
-    // Update _pfc_rdb_bf_seek_total
-    auto bf_seek_total = _statistics->getTickerCount(rocksdb::BLOOM_FILTER_PREFIX_CHECKED);
-    _pfc_rdb_bf_seek_total->set(bf_seek_total);
-    LOG_DEBUG_PREFIX("_pfc_rdb_bf_seek_total: {}", bf_seek_total);
+    GET_TICKER_COUNT_AND_SET_METRIC(BLOOM_FILTER_PREFIX_CHECKED, rdb_bloom_filter_seek_total);
 
-    // Update _pfc_rdb_bf_point_positive_true
-    auto bf_point_positive_true =
-        _statistics->getTickerCount(rocksdb::BLOOM_FILTER_FULL_TRUE_POSITIVE);
-    _pfc_rdb_bf_point_positive_true->set(bf_point_positive_true);
-    LOG_DEBUG_PREFIX("_pfc_rdb_bf_point_positive_true: {}", bf_point_positive_true);
+    GET_TICKER_COUNT_AND_SET_METRIC(BLOOM_FILTER_USEFUL, rdb_bloom_filter_point_lookup_negatives);
 
-    // Update _pfc_rdb_bf_point_positive_total
-    auto bf_point_positive_total = _statistics->getTickerCount(rocksdb::BLOOM_FILTER_FULL_POSITIVE);
-    _pfc_rdb_bf_point_positive_total->set(bf_point_positive_total);
-    LOG_DEBUG_PREFIX("_pfc_rdb_bf_point_positive_total: {}", bf_point_positive_total);
+    GET_TICKER_COUNT_AND_SET_METRIC(BLOOM_FILTER_FULL_POSITIVE,
+                                    rdb_bloom_filter_point_lookup_positives);
 
-    // Update _pfc_rdb_bf_point_negatives
-    auto bf_point_negatives = _statistics->getTickerCount(rocksdb::BLOOM_FILTER_USEFUL);
-    _pfc_rdb_bf_point_negatives->set(bf_point_negatives);
-    LOG_DEBUG_PREFIX("_pfc_rdb_bf_point_negatives: {}", bf_point_negatives);
+    GET_TICKER_COUNT_AND_SET_METRIC(BLOOM_FILTER_FULL_TRUE_POSITIVE,
+                                    rdb_bloom_filter_point_lookup_true_positives);
 
-    // Update _pfc_rdb_block_cache_hit_count and _pfc_rdb_block_cache_total_count
     auto block_cache_hit = _statistics->getTickerCount(rocksdb::BLOCK_CACHE_HIT);
-    _pfc_rdb_block_cache_hit_count->set(block_cache_hit);
-    LOG_DEBUG_PREFIX("_pfc_rdb_block_cache_hit_count: {}", block_cache_hit);
+    METRIC_VAR_SET(rdb_block_cache_hit_count, block_cache_hit);
 
     auto block_cache_miss = _statistics->getTickerCount(rocksdb::BLOCK_CACHE_MISS);
     auto block_cache_total = block_cache_hit + block_cache_miss;
-    _pfc_rdb_block_cache_total_count->set(block_cache_total);
-    LOG_DEBUG_PREFIX("_pfc_rdb_block_cache_total_count: {}", block_cache_total);
+    METRIC_VAR_SET(rdb_block_cache_total_count, block_cache_total);
 
-    // update block memtable/l0/l1/l2andup hit rate under block cache up level
     auto memtable_hit_count = _statistics->getTickerCount(rocksdb::MEMTABLE_HIT);
-    _pfc_rdb_memtable_hit_count->set(memtable_hit_count);
-    LOG_DEBUG_PREFIX("_pfc_rdb_memtable_hit_count: {}", memtable_hit_count);
+    METRIC_VAR_SET(rdb_memtable_hit_count, memtable_hit_count);
 
     auto memtable_miss_count = _statistics->getTickerCount(rocksdb::MEMTABLE_MISS);
     auto memtable_total = memtable_hit_count + memtable_miss_count;
-    _pfc_rdb_memtable_total_count->set(memtable_total);
-    LOG_DEBUG_PREFIX("_pfc_rdb_memtable_total_count: {}", memtable_total);
+    METRIC_VAR_SET(rdb_memtable_total_count, memtable_total);
 
-    auto l0_hit_count = _statistics->getTickerCount(rocksdb::GET_HIT_L0);
-    _pfc_rdb_l0_hit_count->set(l0_hit_count);
-    LOG_DEBUG_PREFIX("_pfc_rdb_l0_hit_count: {}", l0_hit_count);
+    GET_TICKER_COUNT_AND_SET_METRIC(GET_HIT_L0, rdb_l0_hit_count);
 
-    auto l1_hit_count = _statistics->getTickerCount(rocksdb::GET_HIT_L1);
-    _pfc_rdb_l1_hit_count->set(l1_hit_count);
-    LOG_DEBUG_PREFIX("_pfc_rdb_l1_hit_count: {}", l1_hit_count);
+    GET_TICKER_COUNT_AND_SET_METRIC(GET_HIT_L1, rdb_l1_hit_count);
 
-    auto l2andup_hit_count = _statistics->getTickerCount(rocksdb::GET_HIT_L2_AND_UP);
-    _pfc_rdb_l2andup_hit_count->set(l2andup_hit_count);
-    LOG_DEBUG_PREFIX("_pfc_rdb_l2andup_hit_count: {}", l2andup_hit_count);
+    GET_TICKER_COUNT_AND_SET_METRIC(GET_HIT_L2_AND_UP, rdb_l2_and_up_hit_count);
 }
 
 void pegasus_server_impl::update_server_rocksdb_statistics()
diff --git a/src/server/pegasus_server_impl.h b/src/server/pegasus_server_impl.h
index 7203d0ce2..c107089af 100644
--- a/src/server/pegasus_server_impl.h
+++ b/src/server/pegasus_server_impl.h
@@ -539,34 +539,36 @@ private:
     METRIC_VAR_DECLARE_counter(read_expired_values);
     METRIC_VAR_DECLARE_counter(read_filtered_values);
     METRIC_VAR_DECLARE_counter(abnormal_read_requests);
+    METRIC_VAR_DECLARE_counter(throttling_rejected_read_requests);
 
     // rocksdb internal statistics
     // server level
     static ::dsn::perf_counter_wrapper _pfc_rdb_write_limiter_rate_bytes;
     static ::dsn::perf_counter_wrapper _pfc_rdb_block_cache_mem_usage;
-    // replica level
-    dsn::perf_counter_wrapper _pfc_rdb_sst_count;
-    dsn::perf_counter_wrapper _pfc_rdb_sst_size;
-    dsn::perf_counter_wrapper _pfc_rdb_index_and_filter_blocks_mem_usage;
-    dsn::perf_counter_wrapper _pfc_rdb_memtable_mem_usage;
-    dsn::perf_counter_wrapper _pfc_rdb_estimate_num_keys;
-
-    dsn::perf_counter_wrapper _pfc_rdb_bf_seek_negatives;
-    dsn::perf_counter_wrapper _pfc_rdb_bf_seek_total;
-    dsn::perf_counter_wrapper _pfc_rdb_bf_point_positive_true;
-    dsn::perf_counter_wrapper _pfc_rdb_bf_point_positive_total;
-    dsn::perf_counter_wrapper _pfc_rdb_bf_point_negatives;
-    dsn::perf_counter_wrapper _pfc_rdb_block_cache_hit_count;
-    dsn::perf_counter_wrapper _pfc_rdb_block_cache_total_count;
-    dsn::perf_counter_wrapper _pfc_rdb_write_amplification;
-    dsn::perf_counter_wrapper _pfc_rdb_read_amplification;
-    dsn::perf_counter_wrapper _pfc_rdb_memtable_hit_count;
-    dsn::perf_counter_wrapper _pfc_rdb_memtable_total_count;
-    dsn::perf_counter_wrapper _pfc_rdb_l0_hit_count;
-    dsn::perf_counter_wrapper _pfc_rdb_l1_hit_count;
-    dsn::perf_counter_wrapper _pfc_rdb_l2andup_hit_count;
-
-    dsn::perf_counter_wrapper _counter_recent_read_throttling_reject_count;
+
+    // Replica-level metrics for rocksdb.
+    METRIC_VAR_DECLARE_gauge_int64(rdb_total_sst_files);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_total_sst_size_mb);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_estimated_keys);
+
+    METRIC_VAR_DECLARE_gauge_int64(rdb_index_and_filter_blocks_mem_usage_bytes);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_memtable_mem_usage_bytes);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_block_cache_hit_count);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_block_cache_total_count);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_memtable_hit_count);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_memtable_total_count);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_l0_hit_count);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_l1_hit_count);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_l2_and_up_hit_count);
+
+    METRIC_VAR_DECLARE_gauge_int64(rdb_write_amplification);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_read_amplification);
+
+    METRIC_VAR_DECLARE_gauge_int64(rdb_bloom_filter_seek_negatives);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_bloom_filter_seek_total);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_bloom_filter_point_lookup_negatives);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_bloom_filter_point_lookup_positives);
+    METRIC_VAR_DECLARE_gauge_int64(rdb_bloom_filter_point_lookup_true_positives);
 };
 
 } // namespace server
diff --git a/src/server/pegasus_server_impl_init.cpp b/src/server/pegasus_server_impl_init.cpp
index 27aebdfab..0840c094b 100644
--- a/src/server/pegasus_server_impl_init.cpp
+++ b/src/server/pegasus_server_impl_init.cpp
@@ -65,57 +65,179 @@ class replica;
 METRIC_DEFINE_counter(replica,
                       get_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of GET requests for each replica");
+                      "The number of GET requests");
 
 METRIC_DEFINE_counter(replica,
                       multi_get_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of MULTI_GET requests for each replica");
+                      "The number of MULTI_GET requests");
 
 METRIC_DEFINE_counter(replica,
                       batch_get_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of BATCH_GET requests for each replica");
+                      "The number of BATCH_GET requests");
 
 METRIC_DEFINE_counter(replica,
                       scan_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of SCAN requests for each replica");
+                      "The number of SCAN requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                get_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of GET requests for each replica");
+                               "The latency of GET requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                multi_get_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of MULTI_GET requests for each replica");
+                               "The latency of MULTI_GET requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                batch_get_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of BATCH_GET requests for each replica");
+                               "The latency of BATCH_GET requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                scan_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of SCAN requests for each replica");
+                               "The latency of SCAN requests");
 
 METRIC_DEFINE_counter(replica,
                       read_expired_values,
                       dsn::metric_unit::kValues,
-                      "The number of expired values read for each replica");
+                      "The number of expired values read");
 
 METRIC_DEFINE_counter(replica,
                       read_filtered_values,
                       dsn::metric_unit::kValues,
-                      "The number of filtered values read for each replica");
+                      "The number of filtered values read");
 
 METRIC_DEFINE_counter(replica,
                       abnormal_read_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of abnormal read requests for each replica");
+                      "The number of abnormal read requests");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_rejected_read_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected read requests by throttling");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_total_sst_files,
+                          dsn::metric_unit::kFiles,
+                          "The total number of rocksdb sst files");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_total_sst_size_mb,
+                          dsn::metric_unit::kMegaBytes,
+                          "The total size of rocksdb sst files in MB");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_estimated_keys,
+                          dsn::metric_unit::kKeys,
+                          "The estimated number of rocksdb keys");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_index_and_filter_blocks_mem_usage_bytes,
+                          dsn::metric_unit::kBytes,
+                          "The memory usage of rocksdb index and filter blocks in bytes");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_memtable_mem_usage_bytes,
+                          dsn::metric_unit::kBytes,
+                          "The memory usage of rocksdb memtables in bytes");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_block_cache_hit_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The hit number of lookups on rocksdb block cache");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_block_cache_total_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The total number of lookups on rocksdb block cache");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_memtable_hit_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The hit number of lookups on rocksdb memtable");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_memtable_total_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The total number of lookups on rocksdb memtable");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_l0_hit_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The number of lookups served by rocksdb L0");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_l1_hit_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The number of lookups served by rocksdb L1");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_l2_and_up_hit_count,
+                          dsn::metric_unit::kPointLookups,
+                          "The number of lookups served by rocksdb L2 and up");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_write_amplification,
+                          dsn::metric_unit::kAmplification,
+                          "The write amplification of rocksdb");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_read_amplification,
+                          dsn::metric_unit::kAmplification,
+                          "The read amplification of rocksdb");
+
+// Following metrics are rocksdb statistics that are related to bloom filters.
+//
+// To measure prefix bloom filters, these metrics are updated after each ::Seek and ::SeekForPrev if
+// prefix is enabled and check_filter is set:
+// * rdb_bloom_filter_seek_negatives: seek_negatives
+// * rdb_bloom_filter_seek_total: seek_negatives + seek_positives
+//
+// To measure full bloom filters, these metrics are updated after each point lookup. If
+// whole_key_filtering is set, this is the result of checking the bloom of the whole key, otherwise
+// this is the result of checking the bloom of the prefix:
+// * rdb_bloom_filter_point_lookup_negatives: [true] negatives
+// * rdb_bloom_filter_point_lookup_positives: positives
+// * rdb_bloom_filter_point_lookup_true_positives: true positives
+//
+// For details please see https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter#statistic.
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_bloom_filter_seek_negatives,
+                          dsn::metric_unit::kSeeks,
+                          "The number of times the check for prefix bloom filter was useful in "
+                          "avoiding iterator creation (and thus likely IOPs), used by rocksdb for "
+                          "each replica");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_bloom_filter_seek_total,
+                          dsn::metric_unit::kSeeks,
+                          "The number of times prefix bloom filter was checked before creating "
+                          "iterator on a file, used by rocksdb");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_bloom_filter_point_lookup_negatives,
+                          dsn::metric_unit::kPointLookups,
+                          "The number of times full bloom filter has avoided file reads (i.e., "
+                          "negatives), used by rocksdb");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_bloom_filter_point_lookup_positives,
+                          dsn::metric_unit::kPointLookups,
+                          "The number of times full bloom filter has not avoided the reads, used "
+                          "by rocksdb");
+
+METRIC_DEFINE_gauge_int64(replica,
+                          rdb_bloom_filter_point_lookup_true_positives,
+                          dsn::metric_unit::kPointLookups,
+                          "The number of times full bloom filter has not avoided the reads and "
+                          "data actually exist, used by rocksdb");
 
 namespace pegasus {
 namespace server {
@@ -469,7 +591,27 @@ pegasus_server_impl::pegasus_server_impl(dsn::replication::replica *r)
       METRIC_VAR_INIT_replica(scan_latency_ns),
       METRIC_VAR_INIT_replica(read_expired_values),
       METRIC_VAR_INIT_replica(read_filtered_values),
-      METRIC_VAR_INIT_replica(abnormal_read_requests)
+      METRIC_VAR_INIT_replica(abnormal_read_requests),
+      METRIC_VAR_INIT_replica(throttling_rejected_read_requests),
+      METRIC_VAR_INIT_replica(rdb_total_sst_files),
+      METRIC_VAR_INIT_replica(rdb_total_sst_size_mb),
+      METRIC_VAR_INIT_replica(rdb_estimated_keys),
+      METRIC_VAR_INIT_replica(rdb_index_and_filter_blocks_mem_usage_bytes),
+      METRIC_VAR_INIT_replica(rdb_memtable_mem_usage_bytes),
+      METRIC_VAR_INIT_replica(rdb_block_cache_hit_count),
+      METRIC_VAR_INIT_replica(rdb_block_cache_total_count),
+      METRIC_VAR_INIT_replica(rdb_memtable_hit_count),
+      METRIC_VAR_INIT_replica(rdb_memtable_total_count),
+      METRIC_VAR_INIT_replica(rdb_l0_hit_count),
+      METRIC_VAR_INIT_replica(rdb_l1_hit_count),
+      METRIC_VAR_INIT_replica(rdb_l2_and_up_hit_count),
+      METRIC_VAR_INIT_replica(rdb_write_amplification),
+      METRIC_VAR_INIT_replica(rdb_read_amplification),
+      METRIC_VAR_INIT_replica(rdb_bloom_filter_seek_negatives),
+      METRIC_VAR_INIT_replica(rdb_bloom_filter_seek_total),
+      METRIC_VAR_INIT_replica(rdb_bloom_filter_point_lookup_negatives),
+      METRIC_VAR_INIT_replica(rdb_bloom_filter_point_lookup_positives),
+      METRIC_VAR_INIT_replica(rdb_bloom_filter_point_lookup_true_positives)
 {
     _primary_address = dsn::rpc_address(dsn_primary_address()).to_string();
     _gpid = get_gpid();
@@ -675,54 +817,6 @@ pegasus_server_impl::pegasus_server_impl(dsn::replication::replica *r)
     std::string str_gpid = _gpid.to_string();
     char name[256];
 
-    // register the perf counters
-    snprintf(name, 255, "disk.storage.sst.count@%s", str_gpid.c_str());
-    _pfc_rdb_sst_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistic the count of sstable files");
-
-    snprintf(name, 255, "disk.storage.sst(MB)@%s", str_gpid.c_str());
-    _pfc_rdb_sst_size.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistic the size of sstable files");
-
-    snprintf(name, 255, "rdb.block_cache.hit_count@%s", str_gpid.c_str());
-    _pfc_rdb_block_cache_hit_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistic the hit count of rocksdb block cache");
-
-    snprintf(name, 255, "rdb.block_cache.total_count@%s", str_gpid.c_str());
-    _pfc_rdb_block_cache_total_count.init_app_counter(
-        "app.pegasus",
-        name,
-        COUNTER_TYPE_NUMBER,
-        "statistic the total count of rocksdb block cache");
-
-    snprintf(name, 255, "rdb.write_amplification@%s", str_gpid.c_str());
-    _pfc_rdb_write_amplification.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the write amplification of rocksdb");
-
-    snprintf(name, 255, "rdb.read_amplification@%s", str_gpid.c_str());
-    _pfc_rdb_read_amplification.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the read amplification of rocksdb");
-
-    snprintf(name, 255, "rdb.read_memtable_hit_count@%s", str_gpid.c_str());
-    _pfc_rdb_memtable_hit_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the read memtable hit count");
-
-    snprintf(name, 255, "rdb.read_memtable_total_count@%s", str_gpid.c_str());
-    _pfc_rdb_memtable_total_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the read memtable total count");
-
-    snprintf(name, 255, "rdb.read_l0_hit_count@%s", str_gpid.c_str());
-    _pfc_rdb_l0_hit_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the read l0 hit count");
-
-    snprintf(name, 255, "rdb.read_l1_hit_count@%s", str_gpid.c_str());
-    _pfc_rdb_l1_hit_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the read l1 hit count");
-
-    snprintf(name, 255, "rdb.read_l2andup_hit_count@%s", str_gpid.c_str());
-    _pfc_rdb_l2andup_hit_count.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistics the read l2andup hit count");
-
     // These counters are singletons on this server shared by all replicas, so we initialize
     // them only once.
     static std::once_flag flag;
@@ -741,66 +835,7 @@ pegasus_server_impl::pegasus_server_impl(dsn::replication::replica *r)
             COUNTER_TYPE_NUMBER,
             "statistic the through bytes of rocksdb write rate limiter");
     });
-
-    snprintf(name, 255, "rdb.index_and_filter_blocks.memory_usage@%s", str_gpid.c_str());
-    _pfc_rdb_index_and_filter_blocks_mem_usage.init_app_counter(
-        "app.pegasus",
-        name,
-        COUNTER_TYPE_NUMBER,
-        "statistic the memory usage of rocksdb index and filter blocks");
-
-    snprintf(name, 255, "rdb.memtable.memory_usage@%s", str_gpid.c_str());
-    _pfc_rdb_memtable_mem_usage.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistic the memory usage of rocksdb memtable");
-
-    snprintf(name, 255, "rdb.estimate_num_keys@%s", str_gpid.c_str());
-    _pfc_rdb_estimate_num_keys.init_app_counter(
-        "app.pegasus",
-        name,
-        COUNTER_TYPE_NUMBER,
-        "statistics the estimated number of keys inside the rocksdb");
-
-    snprintf(name, 255, "rdb.bf_seek_negatives@%s", str_gpid.c_str());
-    _pfc_rdb_bf_seek_negatives.init_app_counter("app.pegasus",
-                                                name,
-                                                COUNTER_TYPE_NUMBER,
-                                                "statistics the number of times bloom filter was "
-                                                "checked before creating iterator on a file and "
-                                                "useful in avoiding iterator creation (and thus "
-                                                "likely IOPs)");
-
-    snprintf(name, 255, "rdb.bf_seek_total@%s", str_gpid.c_str());
-    _pfc_rdb_bf_seek_total.init_app_counter("app.pegasus",
-                                            name,
-                                            COUNTER_TYPE_NUMBER,
-                                            "statistics the number of times bloom filter was "
-                                            "checked before creating iterator on a file");
-
-    snprintf(name, 255, "rdb.bf_point_positive_true@%s", str_gpid.c_str());
-    _pfc_rdb_bf_point_positive_true.init_app_counter(
-        "app.pegasus",
-        name,
-        COUNTER_TYPE_NUMBER,
-        "statistics the number of times bloom filter has avoided file reads, i.e., negatives");
-
-    snprintf(name, 255, "rdb.bf_point_positive_total@%s", str_gpid.c_str());
-    _pfc_rdb_bf_point_positive_total.init_app_counter(
-        "app.pegasus",
-        name,
-        COUNTER_TYPE_NUMBER,
-        "statistics the number of times bloom FullFilter has not avoided the reads");
-
-    snprintf(name, 255, "rdb.bf_point_negatives@%s", str_gpid.c_str());
-    _pfc_rdb_bf_point_negatives.init_app_counter("app.pegasus",
-                                                 name,
-                                                 COUNTER_TYPE_NUMBER,
-                                                 "statistics the number of times bloom FullFilter "
-                                                 "has not avoided the reads and data actually "
-                                                 "exist");
-
-    auto counter_str = fmt::format("recent.read.throttling.reject.count@{}", str_gpid.c_str());
-    _counter_recent_read_throttling_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
 }
+
 } // namespace server
 } // namespace pegasus
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index da1d056d8..facc83f26 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -161,8 +161,9 @@ class error_code;
     METRIC_VAR_DECLARE(name, dsn::percentile_ptr<int64_t>)
 
 // Initialize a metric variable in user class.
-#define METRIC_VAR_INIT(name, entity) _##name(METRIC_##name.instantiate(entity##_metric_entity()))
-#define METRIC_VAR_INIT_replica(name) METRIC_VAR_INIT(name, replica)
+#define METRIC_VAR_INIT(name, entity, ...)                                                         \
+    _##name(METRIC_##name.instantiate(entity##_metric_entity(), ##__VA_ARGS__))
+#define METRIC_VAR_INIT_replica(name, ...) METRIC_VAR_INIT(name, replica, ##__VA_ARGS__)
 
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
@@ -608,8 +609,15 @@ enum class metric_unit : size_t
     kMicroSeconds,
     kMilliSeconds,
     kSeconds,
+    kBytes,
+    kMegaBytes,
     kRequests,
+    kSeeks,
+    kPointLookups,
     kValues,
+    kKeys,
+    kFiles,
+    kAmplification,
     kInvalidUnit,
 };
 


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


[incubator-pegasus] 05/23: feat(new_metrics): migrate replica-level metrics for capacity_unit_calculator (#1387)

Posted by wa...@apache.org.
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 8d47b50989c06bace2d87879dedc5e7df069e9ba
Author: Dan Wang <wa...@apache.org>
AuthorDate: Tue Mar 14 17:18:35 2023 +0800

    feat(new_metrics): migrate replica-level metrics for capacity_unit_calculator (#1387)
    
    https://github.com/apache/incubator-pegasus/issues/1334
    
    Migrate the metrics in capacity_unit_calculator to new framework, including read/write capacity units
    and the number of bytes consumed by get, multi_get, batch_get, scan, put, multi_put, check_and_set,
    check_and_mutate and backup requests.
---
 src/server/capacity_unit_calculator.cpp | 142 ++++++++++++++++++--------------
 src/server/capacity_unit_calculator.h   |  26 +++---
 src/server/pegasus_write_service.cpp    |  47 +++++------
 src/utils/metrics.h                     |   6 +-
 4 files changed, 118 insertions(+), 103 deletions(-)

diff --git a/src/server/capacity_unit_calculator.cpp b/src/server/capacity_unit_calculator.cpp
index 157577b98..7a62b3d56 100644
--- a/src/server/capacity_unit_calculator.cpp
+++ b/src/server/capacity_unit_calculator.cpp
@@ -35,6 +35,61 @@
 #include "utils/fmt_logging.h"
 #include "utils/token_bucket_throttling_controller.h"
 
+METRIC_DEFINE_counter(replica,
+                      read_capacity_units,
+                      dsn::metric_unit::kCapacityUnits,
+                      "The number of capacity units for read requests");
+
+METRIC_DEFINE_counter(replica,
+                      write_capacity_units,
+                      dsn::metric_unit::kCapacityUnits,
+                      "The number of capacity units for write requests");
+
+METRIC_DEFINE_counter(replica,
+                      get_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for GET requests");
+
+METRIC_DEFINE_counter(replica,
+                      multi_get_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for MULTI_GET requests");
+
+METRIC_DEFINE_counter(replica,
+                      batch_get_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for BATCH_GET requests");
+
+METRIC_DEFINE_counter(replica,
+                      scan_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for SCAN requests");
+
+METRIC_DEFINE_counter(replica,
+                      put_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for PUT requests");
+
+METRIC_DEFINE_counter(replica,
+                      multi_put_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for MULTI_PUT requests");
+
+METRIC_DEFINE_counter(replica,
+                      check_and_set_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for CHECK_AND_SET requests");
+
+METRIC_DEFINE_counter(replica,
+                      check_and_mutate_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for CHECK_AND_MUTATE requests");
+
+METRIC_DEFINE_counter(replica,
+                      backup_request_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The number of bytes for backup requests");
+
 namespace pegasus {
 namespace server {
 
@@ -58,6 +113,17 @@ capacity_unit_calculator::capacity_unit_calculator(
     std::shared_ptr<hotkey_collector> write_hotkey_collector,
     std::shared_ptr<throttling_controller> read_size_throttling_controller)
     : replica_base(r),
+      METRIC_VAR_INIT_replica(read_capacity_units),
+      METRIC_VAR_INIT_replica(write_capacity_units),
+      METRIC_VAR_INIT_replica(get_bytes),
+      METRIC_VAR_INIT_replica(multi_get_bytes),
+      METRIC_VAR_INIT_replica(batch_get_bytes),
+      METRIC_VAR_INIT_replica(scan_bytes),
+      METRIC_VAR_INIT_replica(put_bytes),
+      METRIC_VAR_INIT_replica(multi_put_bytes),
+      METRIC_VAR_INIT_replica(check_and_set_bytes),
+      METRIC_VAR_INIT_replica(check_and_mutate_bytes),
+      METRIC_VAR_INIT_replica(backup_request_bytes),
       _read_hotkey_collector(read_hotkey_collector),
       _write_hotkey_collector(write_hotkey_collector),
       _read_size_throttling_controller(read_size_throttling_controller)
@@ -68,55 +134,6 @@ capacity_unit_calculator::capacity_unit_calculator(
 
     _log_read_cu_size = log(FLAGS_perf_counter_read_capacity_unit_size) / log(2);
     _log_write_cu_size = log(FLAGS_perf_counter_write_capacity_unit_size) / log(2);
-
-    std::string str_gpid = r->get_gpid().to_string();
-    char name[256];
-    snprintf(name, 255, "recent.read.cu@%s", str_gpid.c_str());
-    _pfc_recent_read_cu.init_app_counter("app.pegasus",
-                                         name,
-                                         COUNTER_TYPE_VOLATILE_NUMBER,
-                                         "statistic the recent read capacity units");
-    snprintf(name, 255, "recent.write.cu@%s", str_gpid.c_str());
-    _pfc_recent_write_cu.init_app_counter("app.pegasus",
-                                          name,
-                                          COUNTER_TYPE_VOLATILE_NUMBER,
-                                          "statistic the recent write capacity units");
-
-    snprintf(name, 255, "get_bytes@%s", str_gpid.c_str());
-    _pfc_get_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the get bytes");
-
-    snprintf(name, 255, "multi_get_bytes@%s", str_gpid.c_str());
-    _pfc_multi_get_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the multi get bytes");
-
-    snprintf(name, 255, "batch_get_bytes@%s", str_gpid.c_str());
-    _pfc_batch_get_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the batch get bytes");
-
-    snprintf(name, 255, "scan_bytes@%s", str_gpid.c_str());
-    _pfc_scan_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the scan bytes");
-
-    snprintf(name, 255, "put_bytes@%s", str_gpid.c_str());
-    _pfc_put_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the put bytes");
-
-    snprintf(name, 255, "multi_put_bytes@%s", str_gpid.c_str());
-    _pfc_multi_put_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the multi put bytes");
-
-    snprintf(name, 255, "check_and_set_bytes@%s", str_gpid.c_str());
-    _pfc_check_and_set_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the check and set bytes");
-
-    snprintf(name, 255, "check_and_mutate_bytes@%s", str_gpid.c_str());
-    _pfc_check_and_mutate_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the check and mutate bytes");
-
-    snprintf(name, 255, "backup_request_bytes@%s", str_gpid.c_str());
-    _pfc_backup_request_bytes.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the backup request bytes");
 }
 
 int64_t capacity_unit_calculator::add_read_cu(int64_t read_data_size)
@@ -125,7 +142,7 @@ int64_t capacity_unit_calculator::add_read_cu(int64_t read_data_size)
         read_data_size > 0
             ? (read_data_size + FLAGS_perf_counter_read_capacity_unit_size - 1) >> _log_read_cu_size
             : 1;
-    _pfc_recent_read_cu->add(read_cu);
+    METRIC_VAR_INCREMENT_BY(read_capacity_units, read_cu);
     _read_size_throttling_controller->consume_token(read_data_size);
     return read_cu;
 }
@@ -136,7 +153,7 @@ int64_t capacity_unit_calculator::add_write_cu(int64_t write_data_size)
                            ? (write_data_size + FLAGS_perf_counter_write_capacity_unit_size - 1) >>
                                  _log_write_cu_size
                            : 1;
-    _pfc_recent_write_cu->add(write_cu);
+    METRIC_VAR_INCREMENT_BY(write_capacity_units, write_cu);
     return write_cu;
 }
 
@@ -146,7 +163,7 @@ void capacity_unit_calculator::add_get_cu(dsn::message_ex *req,
                                           const dsn::blob &value)
 {
     auto total_size = key.size() + value.size();
-    _pfc_get_bytes->add(total_size);
+    METRIC_VAR_INCREMENT_BY(get_bytes, total_size);
     add_backup_request_bytes(req, total_size);
     if (status != rocksdb::Status::kOk && status != rocksdb::Status::kNotFound) {
         return;
@@ -173,7 +190,7 @@ void capacity_unit_calculator::add_multi_get_cu(dsn::message_ex *req,
         data_size += hash_key.size() + kv.key.size() + kv.value.size();
     }
     auto total_size = hash_key.size() + multi_get_bytes;
-    _pfc_multi_get_bytes->add(total_size);
+    METRIC_VAR_INCREMENT_BY(multi_get_bytes, total_size);
     add_backup_request_bytes(req, total_size);
 
     if (status != rocksdb::Status::kOk && status != rocksdb::Status::kNotFound &&
@@ -201,7 +218,7 @@ void capacity_unit_calculator::add_batch_get_cu(dsn::message_ex *req,
         _read_hotkey_collector->capture_hash_key(data.hash_key, 1);
     }
 
-    _pfc_batch_get_bytes->add(data_size);
+    METRIC_VAR_INCREMENT_BY(batch_get_bytes, data_size);
     add_backup_request_bytes(req, data_size);
 
     if (status != rocksdb::Status::kOk && status != rocksdb::Status::kNotFound &&
@@ -237,7 +254,7 @@ void capacity_unit_calculator::add_scan_cu(dsn::message_ex *req,
         data_size += kv.key.size() + kv.value.size();
     }
     add_read_cu(data_size);
-    _pfc_scan_bytes->add(data_size);
+    METRIC_VAR_INCREMENT_BY(scan_bytes, data_size);
     add_backup_request_bytes(req, data_size);
 }
 
@@ -269,7 +286,7 @@ void capacity_unit_calculator::add_put_cu(int32_t status,
                                           const dsn::blob &key,
                                           const dsn::blob &value)
 {
-    _pfc_put_bytes->add(key.size() + value.size());
+    METRIC_VAR_INCREMENT_BY(put_bytes, key.size() + value.size());
     if (status != rocksdb::Status::kOk) {
         return;
     }
@@ -296,7 +313,7 @@ void capacity_unit_calculator::add_multi_put_cu(int32_t status,
         multi_put_bytes += kv.key.size() + kv.value.size();
         data_size += hash_key.size() + kv.key.size() + kv.value.size();
     }
-    _pfc_multi_put_bytes->add(hash_key.size() + multi_put_bytes);
+    METRIC_VAR_INCREMENT_BY(multi_put_bytes, hash_key.size() + multi_put_bytes);
     uint64_t key_count = kvs.size();
     _write_hotkey_collector->capture_hash_key(hash_key, key_count);
 
@@ -343,8 +360,9 @@ void capacity_unit_calculator::add_check_and_set_cu(int32_t status,
                                                     const dsn::blob &value)
 {
 
-    _pfc_check_and_set_bytes->add(hash_key.size() + check_sort_key.size() + set_sort_key.size() +
-                                  value.size());
+    METRIC_VAR_INCREMENT_BY(check_and_set_bytes,
+                            hash_key.size() + check_sort_key.size() + set_sort_key.size() +
+                                value.size());
     if (status != rocksdb::Status::kOk && status != rocksdb::Status::kInvalidArgument &&
         status != rocksdb::Status::kTryAgain) {
         return;
@@ -370,8 +388,8 @@ void capacity_unit_calculator::add_check_and_mutate_cu(
         check_and_mutate_bytes += m.sort_key.size() + m.value.size();
         data_size += hash_key.size() + m.sort_key.size() + m.value.size();
     }
-    _pfc_check_and_mutate_bytes->add(hash_key.size() + check_sort_key.size() +
-                                     check_and_mutate_bytes);
+    METRIC_VAR_INCREMENT_BY(check_and_mutate_bytes,
+                            hash_key.size() + check_sort_key.size() + check_and_mutate_bytes);
 
     if (status != rocksdb::Status::kOk && status != rocksdb::Status::kInvalidArgument &&
         status != rocksdb::Status::kTryAgain) {
@@ -389,7 +407,7 @@ void capacity_unit_calculator::add_check_and_mutate_cu(
 void capacity_unit_calculator::add_backup_request_bytes(dsn::message_ex *req, int64_t bytes)
 {
     if (req->is_backup_request()) {
-        _pfc_backup_request_bytes->add(bytes);
+        METRIC_VAR_INCREMENT_BY(backup_request_bytes, bytes);
     }
 }
 
diff --git a/src/server/capacity_unit_calculator.h b/src/server/capacity_unit_calculator.h
index 0b91ea4d2..6d30a07ef 100644
--- a/src/server/capacity_unit_calculator.h
+++ b/src/server/capacity_unit_calculator.h
@@ -108,18 +108,20 @@ private:
     uint32_t _log_read_cu_size;
     uint32_t _log_write_cu_size;
 
-    ::dsn::perf_counter_wrapper _pfc_recent_read_cu;
-    ::dsn::perf_counter_wrapper _pfc_recent_write_cu;
-
-    ::dsn::perf_counter_wrapper _pfc_get_bytes;
-    ::dsn::perf_counter_wrapper _pfc_multi_get_bytes;
-    ::dsn::perf_counter_wrapper _pfc_batch_get_bytes;
-    ::dsn::perf_counter_wrapper _pfc_scan_bytes;
-    ::dsn::perf_counter_wrapper _pfc_put_bytes;
-    ::dsn::perf_counter_wrapper _pfc_multi_put_bytes;
-    ::dsn::perf_counter_wrapper _pfc_check_and_set_bytes;
-    ::dsn::perf_counter_wrapper _pfc_check_and_mutate_bytes;
-    ::dsn::perf_counter_wrapper _pfc_backup_request_bytes;
+    METRIC_VAR_DECLARE_counter(read_capacity_units);
+    METRIC_VAR_DECLARE_counter(write_capacity_units);
+
+    METRIC_VAR_DECLARE_counter(get_bytes);
+    METRIC_VAR_DECLARE_counter(multi_get_bytes);
+    METRIC_VAR_DECLARE_counter(batch_get_bytes);
+    METRIC_VAR_DECLARE_counter(scan_bytes);
+
+    METRIC_VAR_DECLARE_counter(put_bytes);
+    METRIC_VAR_DECLARE_counter(multi_put_bytes);
+    METRIC_VAR_DECLARE_counter(check_and_set_bytes);
+    METRIC_VAR_DECLARE_counter(check_and_mutate_bytes);
+
+    METRIC_VAR_DECLARE_counter(backup_request_bytes);
 
     /*
         hotkey capturing weight rules:
diff --git a/src/server/pegasus_write_service.cpp b/src/server/pegasus_write_service.cpp
index 4889329d9..969e7d141 100644
--- a/src/server/pegasus_write_service.cpp
+++ b/src/server/pegasus_write_service.cpp
@@ -55,83 +55,82 @@ class message_ex;
 METRIC_DEFINE_counter(replica,
                       put_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of PUT requests for each replica");
+                      "The number of PUT requests");
 
 METRIC_DEFINE_counter(replica,
                       multi_put_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of MULTI_PUT requests for each replica");
+                      "The number of MULTI_PUT requests");
 
 METRIC_DEFINE_counter(replica,
                       remove_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of REMOVE requests for each replica");
+                      "The number of REMOVE requests");
 
 METRIC_DEFINE_counter(replica,
                       multi_remove_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of MULTI_REMOVE requests for each replica");
+                      "The number of MULTI_REMOVE requests");
 
 METRIC_DEFINE_counter(replica,
                       incr_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of INCR requests for each replica");
+                      "The number of INCR requests");
 
 METRIC_DEFINE_counter(replica,
                       check_and_set_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of CHECK_AND_SET requests for each replica");
+                      "The number of CHECK_AND_SET requests");
 
 METRIC_DEFINE_counter(replica,
                       check_and_mutate_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of CHECK_AND_MUTATE requests for each replica");
+                      "The number of CHECK_AND_MUTATE requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                put_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of PUT requests for each replica");
+                               "The latency of PUT requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                multi_put_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of MULTI_PUT requests for each replica");
+                               "The latency of MULTI_PUT requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                remove_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of REMOVE requests for each replica");
+                               "The latency of REMOVE requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                multi_remove_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of MULTI_REMOVE requests for each replica");
+                               "The latency of MULTI_REMOVE requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                incr_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of INCR requests for each replica");
+                               "The latency of INCR requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                check_and_set_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of CHECK_AND_SET requests for each replica");
+                               "The latency of CHECK_AND_SET requests");
 
 METRIC_DEFINE_percentile_int64(replica,
                                check_and_mutate_latency_ns,
                                dsn::metric_unit::kNanoSeconds,
-                               "The latency of CHECK_AND_MUTATE requests for each replica");
+                               "The latency of CHECK_AND_MUTATE requests");
 
 METRIC_DEFINE_counter(replica,
                       dup_requests,
                       dsn::metric_unit::kRequests,
-                      "The number of DUPLICATE requests for each replica");
+                      "The number of DUPLICATE requests");
 
-METRIC_DEFINE_percentile_int64(
-    replica,
-    dup_time_lag_ms,
-    dsn::metric_unit::kMilliSeconds,
-    "the time lag (in ms) between master and slave in the duplication for each replica");
+METRIC_DEFINE_percentile_int64(replica,
+                               dup_time_lag_ms,
+                               dsn::metric_unit::kMilliSeconds,
+                               "the time lag (in ms) between master and slave in the duplication");
 
 METRIC_DEFINE_counter(
     replica,
@@ -176,12 +175,6 @@ pegasus_write_service::pegasus_write_service(pegasus_server_impl *server)
       _put_batch_size(0),
       _remove_batch_size(0)
 {
-    _dup_lagging_write_threshold_ms = dsn_config_get_value_int64(
-        "pegasus.server",
-        "dup_lagging_write_threshold_ms",
-        10 * 1000,
-        "If the duration that a write flows from master to slave is larger than this threshold, "
-        "the write is defined a lagging write.");
 }
 
 pegasus_write_service::~pegasus_write_service() {}
@@ -372,7 +365,7 @@ int pegasus_write_service::duplicate(int64_t decree,
         METRIC_VAR_INCREMENT(dup_requests);
         METRIC_VAR_AUTO_LATENCY(
             dup_time_lag_ms, request.timestamp * 1000, [this](uint64_t latency) {
-                if (latency > _dup_lagging_write_threshold_ms) {
+                if (latency > FLAGS_dup_lagging_write_threshold_ms) {
                     METRIC_VAR_INCREMENT(dup_lagging_writes);
                 }
             });
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index facc83f26..333534703 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -168,8 +168,9 @@ class error_code;
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
     do {                                                                                           \
-        if (x != 0) {                                                                              \
-            _##name->increment_by(x);                                                              \
+        const auto v = (x);                                                                        \
+        if (v != 0) {                                                                              \
+            _##name->increment_by(v);                                                              \
         }                                                                                          \
     } while (0)
 
@@ -611,6 +612,7 @@ enum class metric_unit : size_t
     kSeconds,
     kBytes,
     kMegaBytes,
+    kCapacityUnits,
     kRequests,
     kSeeks,
     kPointLookups,


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


[incubator-pegasus] 08/23: feat(new_metrics): migrate replica-level metrics for pegasus_mutation_duplicator (#1413)

Posted by wa...@apache.org.
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 b9c541ffe9dd3d88a595fe596ef9fc8fa8a81038
Author: Dan Wang <wa...@apache.org>
AuthorDate: Fri Mar 24 16:06:02 2023 +0800

    feat(new_metrics): migrate replica-level metrics for pegasus_mutation_duplicator (#1413)
    
    https://github.com/apache/incubator-pegasus/issues/1412
    
    Migrate replica-level metrics in pegasus_mutation_duplicator class to new
    framework, including the numbers of successful/failed DUPLICATE requests
    sent from mutation duplicator.
---
 src/server/pegasus_mutation_duplicator.cpp | 36 ++++++++++++++----------------
 src/server/pegasus_mutation_duplicator.h   |  6 ++---
 2 files changed, 20 insertions(+), 22 deletions(-)

diff --git a/src/server/pegasus_mutation_duplicator.cpp b/src/server/pegasus_mutation_duplicator.cpp
index e9295af35..8d87ce60b 100644
--- a/src/server/pegasus_mutation_duplicator.cpp
+++ b/src/server/pegasus_mutation_duplicator.cpp
@@ -19,14 +19,11 @@
 
 #include "pegasus_mutation_duplicator.h"
 
-#include <fmt/core.h>
-#include <fmt/ostream.h>
 #include <pegasus/error.h>
 #include <sys/types.h>
 #include <chrono>
 #include <cstdint>
 #include <functional>
-#include <iosfwd>
 #include <memory>
 #include <tuple>
 #include <utility>
@@ -34,16 +31,15 @@
 
 #include "client_lib/pegasus_client_impl.h"
 #include "common/duplication_common.h"
-#include "common/gpid.h"
 #include "duplication_internal_types.h"
 #include "pegasus/client.h"
 #include "pegasus_key_schema.h"
-#include "perf_counter/perf_counter.h"
 #include "rrdb/rrdb.code.definition.h"
 #include "rrdb/rrdb_types.h"
 #include "runtime/message_utils.h"
 #include "runtime/rpc/rpc_message.h"
 #include "server/pegasus_write_service.h"
+#include "utils/autoref_ptr.h"
 #include "utils/blob.h"
 #include "utils/chrono_literals.h"
 #include "utils/error_code.h"
@@ -51,6 +47,16 @@
 #include "utils/fmt_logging.h"
 #include "utils/rand.h"
 
+METRIC_DEFINE_counter(replica,
+                      successful_mutation_dup_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of successful DUPLICATE requests sent from mutation duplicator");
+
+METRIC_DEFINE_counter(replica,
+                      failed_mutation_dup_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of failed DUPLICATE requests sent from mutation duplicator");
+
 namespace dsn {
 namespace replication {
 struct replica_base;
@@ -99,7 +105,10 @@ using namespace dsn::literals::chrono_literals;
 pegasus_mutation_duplicator::pegasus_mutation_duplicator(dsn::replication::replica_base *r,
                                                          dsn::string_view remote_cluster,
                                                          dsn::string_view app)
-    : mutation_duplicator(r), _remote_cluster(remote_cluster)
+    : mutation_duplicator(r),
+      _remote_cluster(remote_cluster),
+      METRIC_VAR_INIT_replica(successful_mutation_dup_requests),
+      METRIC_VAR_INIT_replica(failed_mutation_dup_requests)
 {
     // initialize pegasus-client when this class is first time used.
     static __attribute__((unused)) bool _dummy = pegasus_client_factory::initialize(nullptr);
@@ -123,17 +132,6 @@ pegasus_mutation_duplicator::pegasus_mutation_duplicator(dsn::replication::repli
     // never possible to duplicate data to itself
     CHECK_NE_PREFIX_MSG(
         get_current_cluster_id(), _remote_cluster_id, "invalid remote cluster: {}", remote_cluster);
-
-    std::string str_gpid = fmt::format("{}", get_gpid());
-    _shipped_ops.init_app_counter("app.pegasus",
-                                  fmt::format("dup_shipped_ops@{}", str_gpid).c_str(),
-                                  COUNTER_TYPE_RATE,
-                                  "the total ops of DUPLICATE requests sent from this app");
-    _failed_shipping_ops.init_app_counter(
-        "app.pegasus",
-        fmt::format("dup_failed_shipping_ops@{}", str_gpid).c_str(),
-        COUNTER_TYPE_RATE,
-        "the qps of failed DUPLICATE requests sent from this app");
 }
 
 void pegasus_mutation_duplicator::send(uint64_t hash, callback cb)
@@ -164,7 +162,7 @@ void pegasus_mutation_duplicator::on_duplicate_reply(uint64_t hash,
     }
 
     if (perr != PERR_OK || err != dsn::ERR_OK) {
-        _failed_shipping_ops->increment();
+        METRIC_VAR_INCREMENT(failed_mutation_dup_requests);
 
         // randomly log the 1% of the failed duplicate rpc, because minor number of
         // errors are acceptable.
@@ -177,7 +175,7 @@ void pegasus_mutation_duplicator::on_duplicate_reply(uint64_t hash,
         // duplicating an illegal write to server is unacceptable, fail fast.
         CHECK_NE_PREFIX_MSG(perr, PERR_INVALID_ARGUMENT, rpc.response().error_hint);
     } else {
-        _shipped_ops->increment();
+        METRIC_VAR_INCREMENT(successful_mutation_dup_requests);
         _total_shipped_size +=
             rpc.dsn_request()->header->body_length + rpc.dsn_request()->header->hdr_length;
     }
diff --git a/src/server/pegasus_mutation_duplicator.h b/src/server/pegasus_mutation_duplicator.h
index 2107cda5b..9a5aa086c 100644
--- a/src/server/pegasus_mutation_duplicator.h
+++ b/src/server/pegasus_mutation_duplicator.h
@@ -25,13 +25,13 @@
 #include <map>
 #include <string>
 
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/duplication/mutation_duplicator.h"
 #include "rrdb/rrdb.client.h"
 #include "runtime/pipeline.h"
 #include "runtime/task/task_code.h"
 #include "runtime/task/task_tracker.h"
 #include "utils/chrono_literals.h"
+#include "utils/metrics.h"
 #include "utils/string_view.h"
 #include "utils/zlocks.h"
 
@@ -89,8 +89,8 @@ private:
 
     size_t _total_shipped_size{0};
 
-    dsn::perf_counter_wrapper _shipped_ops;
-    dsn::perf_counter_wrapper _failed_shipping_ops;
+    METRIC_VAR_DECLARE_counter(successful_mutation_dup_requests);
+    METRIC_VAR_DECLARE_counter(failed_mutation_dup_requests);
 };
 
 // Decodes the binary `request_data` into write request in thrift struct, and


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


[incubator-pegasus] 01/23: feat(new_metrics): add replica-level metric entity (#1345)

Posted by wa...@apache.org.
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 09b5d5926174faf8e40ff1acca8ed58ef2c4854f
Author: Dan Wang <wa...@apache.org>
AuthorDate: Mon Feb 13 15:45:40 2023 +0800

    feat(new_metrics): add replica-level metric entity (#1345)
---
 src/replica/replica_base.cpp | 52 ++++++++++++++++++++++++++++++++++++++++++++
 src/replica/replica_base.h   | 10 +++++----
 2 files changed, 58 insertions(+), 4 deletions(-)

diff --git a/src/replica/replica_base.cpp b/src/replica/replica_base.cpp
new file mode 100644
index 000000000..11e08ae05
--- /dev/null
+++ b/src/replica/replica_base.cpp
@@ -0,0 +1,52 @@
+// 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 "replica_base.h"
+
+#include <fmt/core.h>
+
+METRIC_DEFINE_entity(replica);
+
+namespace dsn {
+namespace replication {
+
+namespace {
+
+metric_entity_ptr instantiate_replica_metric_entity(const gpid &id)
+{
+    auto entity_id = fmt::format("replica_{}", id);
+
+    // Do NOT add `replica_base._app_name` as the table name to the attributes of entity, since
+    // it is read-only and will never be updated even if the table is renamed.
+    return METRIC_ENTITY_replica.instantiate(
+        entity_id,
+        {{"table_id", std::to_string(id.get_app_id())},
+         {"partition_id", std::to_string(id.get_partition_index())}});
+}
+
+} // anonymous namespace
+
+replica_base::replica_base(gpid id, string_view name, string_view app_name)
+    : _gpid(id),
+      _name(name),
+      _app_name(app_name),
+      _replica_metric_entity(instantiate_replica_metric_entity(id))
+{
+}
+
+} // namespace replication
+} // namespace dsn
diff --git a/src/replica/replica_base.h b/src/replica/replica_base.h
index 64f294e87..88202d055 100644
--- a/src/replica/replica_base.h
+++ b/src/replica/replica_base.h
@@ -27,6 +27,7 @@
 #pragma once
 
 #include "common/gpid.h"
+#include "utils/metrics.h"
 #include "utils/string_view.h"
 
 namespace dsn {
@@ -35,10 +36,7 @@ namespace replication {
 /// Base class for types that are one-instance-per-replica.
 struct replica_base
 {
-    replica_base(gpid id, string_view name, string_view app_name)
-        : _gpid(id), _name(name), _app_name(app_name)
-    {
-    }
+    replica_base(gpid id, string_view name, string_view app_name);
 
     explicit replica_base(replica_base *rhs)
         : replica_base(rhs->get_gpid(), rhs->replica_name(), rhs->_app_name)
@@ -53,10 +51,14 @@ struct replica_base
 
     const char *log_prefix() const { return _name.c_str(); }
 
+    const metric_entity_ptr &replica_metric_entity() const { return _replica_metric_entity; }
+
 private:
     const gpid _gpid;
     const std::string _name;
+    // TODO(wangdan): drop `_app_name` or make it changeable, since a table could be renamed.
     const std::string _app_name;
+    const metric_entity_ptr _replica_metric_entity;
 };
 
 } // namespace replication


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


[incubator-pegasus] 12/23: feat(new_metrics): add disk-level metric entity and migrate disk-level metrics for fs_manager (#1427)

Posted by wa...@apache.org.
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 8a8dbc24f6d5bd797bf669258ae6e2ec6e895088
Author: Dan Wang <wa...@apache.org>
AuthorDate: Thu Apr 6 11:41:34 2023 +0800

    feat(new_metrics): add disk-level metric entity and migrate disk-level metrics for fs_manager (#1427)
    
    https://github.com/apache/incubator-pegasus/issues/1425
    
    In perf counters, all metrics of `fs_manager` are server-level. For example,
    the total capacity and the available capacity of all disks where there are
    data of pegasus.
    
    However, sometimes the capacity and the available capacity of each disk
    seem more important: no space left on the disk will lead to serious problems.
    Therefore, after being migrated to new framework, the server-level metrics
    of perf counters become disk-level, including the capacity and the available
    capacity of a disk. As for another disk-level metric -- the available percentage
    of each disk used by a replica server, just use division operator.
    
    Once server-level metrics are needed, just aggregate on the disk-level ones.
    To compute another 2 server-level metrics -- the minimal/maximal available
    percentage among all disks used by a replica server in a node, for example,
    just use min/max operators over disk-level ones for Prometheus.
    
    To implement disk-level metrics, disk-level metric entity are also added.
---
 src/common/fs_manager.cpp      | 102 +++++++++++++++++++++++++----------------
 src/common/fs_manager.h        |  58 +++++++++++++----------
 src/common/test/CMakeLists.txt |   1 +
 src/meta/test/misc/misc.cpp    |   2 +-
 src/replica/replica_stub.cpp   |   1 -
 src/utils/metrics.h            |   7 +++
 6 files changed, 105 insertions(+), 66 deletions(-)

diff --git a/src/common/fs_manager.cpp b/src/common/fs_manager.cpp
index 5a460a896..61ca99411 100644
--- a/src/common/fs_manager.cpp
+++ b/src/common/fs_manager.cpp
@@ -41,7 +41,7 @@
 
 #include "common/gpid.h"
 #include "common/replication_enums.h"
-#include "perf_counter/perf_counter.h"
+#include "fmt/core.h"
 #include "runtime/api_layer1.h"
 #include "runtime/rpc/rpc_address.h"
 #include "utils/fail_point.h"
@@ -49,6 +49,18 @@
 #include "utils/fmt_logging.h"
 #include "utils/string_view.h"
 
+METRIC_DEFINE_entity(disk);
+
+METRIC_DEFINE_gauge_int64(disk,
+                          total_disk_capacity_mb,
+                          dsn::metric_unit::kMegaBytes,
+                          "The total disk capacity");
+
+METRIC_DEFINE_gauge_int64(disk,
+                          avail_disk_capacity_mb,
+                          dsn::metric_unit::kMegaBytes,
+                          "The available disk capacity");
+
 namespace dsn {
 namespace replication {
 
@@ -61,6 +73,34 @@ DSN_DEFINE_int32(replication,
                  "space insufficient");
 DSN_TAG_VARIABLE(disk_min_available_space_ratio, FT_MUTABLE);
 
+namespace {
+
+metric_entity_ptr instantiate_disk_metric_entity(const std::string &tag,
+                                                 const std::string &data_dir)
+{
+    auto entity_id = fmt::format("disk_{}", tag);
+
+    return METRIC_ENTITY_disk.instantiate(entity_id, {{"tag", tag}, {"data_dir", data_dir}});
+}
+
+} // anonymous namespace
+
+disk_capacity_metrics::disk_capacity_metrics(const std::string &tag, const std::string &data_dir)
+    : _disk_metric_entity(instantiate_disk_metric_entity(tag, data_dir)),
+      METRIC_VAR_INIT_disk(total_disk_capacity_mb),
+      METRIC_VAR_INIT_disk(avail_disk_capacity_mb)
+{
+}
+
+const metric_entity_ptr &disk_capacity_metrics::disk_metric_entity() const
+{
+    CHECK_NOTNULL(_disk_metric_entity,
+                  "disk metric entity should has been instantiated: "
+                  "uninitialized entity cannot be used to instantiate "
+                  "metric");
+    return _disk_metric_entity;
+}
+
 unsigned dir_node::replicas_count() const
 {
     unsigned sum = 0;
@@ -108,6 +148,9 @@ bool dir_node::update_disk_stat(const bool update_disk_status)
     disk_available_ratio = static_cast<int>(
         disk_capacity_mb == 0 ? 0 : std::round(disk_available_mb * 100.0 / disk_capacity_mb));
 
+    METRIC_CALL_SET_METHOD(disk_capacity, total_disk_capacity_mb, disk_capacity_mb);
+    METRIC_CALL_SET_METHOD(disk_capacity, avail_disk_capacity_mb, disk_available_mb);
+
     if (!update_disk_status) {
         LOG_INFO("update disk space succeed: dir = {}, capacity_mb = {}, available_mb = {}, "
                  "available_ratio = {}%",
@@ -134,32 +177,6 @@ bool dir_node::update_disk_stat(const bool update_disk_status)
     return (old_status != new_status);
 }
 
-fs_manager::fs_manager(bool for_test)
-{
-    if (!for_test) {
-        _counter_total_capacity_mb.init_app_counter("eon.replica_stub",
-                                                    "disk.capacity.total(MB)",
-                                                    COUNTER_TYPE_NUMBER,
-                                                    "total disk capacity in MB");
-        _counter_total_available_mb.init_app_counter("eon.replica_stub",
-                                                     "disk.available.total(MB)",
-                                                     COUNTER_TYPE_NUMBER,
-                                                     "total disk available in MB");
-        _counter_total_available_ratio.init_app_counter("eon.replica_stub",
-                                                        "disk.available.total.ratio",
-                                                        COUNTER_TYPE_NUMBER,
-                                                        "total disk available ratio");
-        _counter_min_available_ratio.init_app_counter("eon.replica_stub",
-                                                      "disk.available.min.ratio",
-                                                      COUNTER_TYPE_NUMBER,
-                                                      "minimal disk available ratio in all disks");
-        _counter_max_available_ratio.init_app_counter("eon.replica_stub",
-                                                      "disk.available.max.ratio",
-                                                      COUNTER_TYPE_NUMBER,
-                                                      "maximal disk available ratio in all disks");
-    }
-}
-
 dir_node *fs_manager::get_dir_node(const std::string &subdir) const
 {
     zauto_read_lock l(_lock);
@@ -298,17 +315,29 @@ bool fs_manager::for_each_dir_node(const std::function<bool(const dir_node &)> &
 
 void fs_manager::update_disk_stat(bool check_status_changed)
 {
-    reset_disk_stat();
+    _total_capacity_mb = 0;
+    _total_available_mb = 0;
+    int total_available_ratio = 0;
+    int min_available_ratio = 100;
+    int max_available_ratio = 0;
+
+    // _status_updated_dir_nodes is accessed sequentially in update_disk_stat() and
+    // replica_stub::update_disks_status() during replica_stub::on_disk_stat(), thus
+    // no need to protect it by lock.
+    _status_updated_dir_nodes.clear();
+
+    zauto_read_lock l(_lock);
+
     for (auto &dir_node : _dir_nodes) {
         if (dir_node->update_disk_stat(check_status_changed)) {
             _status_updated_dir_nodes.emplace_back(dir_node);
         }
         _total_capacity_mb += dir_node->disk_capacity_mb;
         _total_available_mb += dir_node->disk_available_mb;
-        _min_available_ratio = std::min(dir_node->disk_available_ratio, _min_available_ratio);
-        _max_available_ratio = std::max(dir_node->disk_available_ratio, _max_available_ratio);
+        min_available_ratio = std::min(dir_node->disk_available_ratio, min_available_ratio);
+        max_available_ratio = std::max(dir_node->disk_available_ratio, max_available_ratio);
     }
-    _total_available_ratio = static_cast<int>(
+    total_available_ratio = static_cast<int>(
         _total_capacity_mb == 0 ? 0 : std::round(_total_available_mb * 100.0 / _total_capacity_mb));
 
     LOG_INFO("update disk space succeed: disk_count = {}, total_capacity_mb = {}, "
@@ -317,14 +346,9 @@ void fs_manager::update_disk_stat(bool check_status_changed)
              _dir_nodes.size(),
              _total_capacity_mb,
              _total_available_mb,
-             _total_available_ratio,
-             _min_available_ratio,
-             _max_available_ratio);
-    _counter_total_capacity_mb->set(_total_capacity_mb);
-    _counter_total_available_mb->set(_total_available_mb);
-    _counter_total_available_ratio->set(_total_available_ratio);
-    _counter_min_available_ratio->set(_min_available_ratio);
-    _counter_max_available_ratio->set(_max_available_ratio);
+             total_available_ratio,
+             min_available_ratio,
+             max_available_ratio);
 }
 
 void fs_manager::add_new_dir_node(const std::string &data_dir, const std::string &tag)
diff --git a/src/common/fs_manager.h b/src/common/fs_manager.h
index 75427cc89..6efe1b1bd 100644
--- a/src/common/fs_manager.h
+++ b/src/common/fs_manager.h
@@ -28,9 +28,11 @@
 
 #include "common/replication_other_types.h"
 #include "metadata_types.h"
-#include "perf_counter/perf_counter_wrapper.h"
+#include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/flags.h"
+#include "utils/metrics.h"
+#include "utils/ports.h"
 #include "utils/zlocks.h"
 
 namespace dsn {
@@ -41,6 +43,25 @@ class replication_options;
 
 DSN_DECLARE_int32(disk_min_available_space_ratio);
 
+class disk_capacity_metrics
+{
+public:
+    disk_capacity_metrics(const std::string &tag, const std::string &data_dir);
+    ~disk_capacity_metrics() = default;
+
+    const metric_entity_ptr &disk_metric_entity() const;
+
+    METRIC_DEFINE_SET_METHOD(total_disk_capacity_mb, int64_t)
+    METRIC_DEFINE_SET_METHOD(avail_disk_capacity_mb, int64_t)
+
+private:
+    const metric_entity_ptr _disk_metric_entity;
+    METRIC_VAR_DECLARE_gauge_int64(total_disk_capacity_mb);
+    METRIC_VAR_DECLARE_gauge_int64(avail_disk_capacity_mb);
+
+    DISALLOW_COPY_AND_ASSIGN(disk_capacity_metrics);
+};
+
 struct dir_node
 {
 public:
@@ -54,6 +75,9 @@ public:
     std::map<app_id, std::set<gpid>> holding_primary_replicas;
     std::map<app_id, std::set<gpid>> holding_secondary_replicas;
 
+private:
+    disk_capacity_metrics disk_capacity;
+
 public:
     dir_node(const std::string &tag_,
              const std::string &dir_,
@@ -66,7 +90,8 @@ public:
           disk_capacity_mb(disk_capacity_mb_),
           disk_available_mb(disk_available_mb_),
           disk_available_ratio(disk_available_ratio_),
-          status(status_)
+          status(status_),
+          disk_capacity(tag_, dir_)
     {
     }
     unsigned replicas_count(app_id id) const;
@@ -79,8 +104,8 @@ public:
 class fs_manager
 {
 public:
-    fs_manager(bool for_test);
-    ~fs_manager() {}
+    fs_manager() = default;
+    ~fs_manager() = default;
 
     // this should be called before open/load any replicas
     dsn::error_code initialize(const replication_options &opts);
@@ -106,27 +131,16 @@ public:
     }
 
 private:
-    void reset_disk_stat()
-    {
-        _total_capacity_mb = 0;
-        _total_available_mb = 0;
-        _total_available_ratio = 0;
-        _min_available_ratio = 100;
-        _max_available_ratio = 0;
-        _status_updated_dir_nodes.clear();
-    }
-
     dir_node *get_dir_node(const std::string &subdir) const;
 
-    // when visit the tag/storage of the _dir_nodes map, there's no need to protect by the lock.
-    // but when visit the holding_replicas, you must take care.
+    // TODO(wangdan): _dir_nodes should be protected by lock since add_new_disk are supported:
+    // it might be updated arbitrarily at any time.
+    //
+    // Especially when visiting the holding_replicas, you must take care.
     mutable zrwlock_nr _lock;
 
     int64_t _total_capacity_mb = 0;
     int64_t _total_available_mb = 0;
-    int _total_available_ratio = 0;
-    int _min_available_ratio = 100;
-    int _max_available_ratio = 0;
 
     std::vector<std::shared_ptr<dir_node>> _dir_nodes;
     std::vector<std::string> _available_data_dirs;
@@ -136,12 +150,6 @@ private:
     // in this round
     std::vector<std::shared_ptr<dir_node>> _status_updated_dir_nodes;
 
-    perf_counter_wrapper _counter_total_capacity_mb;
-    perf_counter_wrapper _counter_total_available_mb;
-    perf_counter_wrapper _counter_total_available_ratio;
-    perf_counter_wrapper _counter_min_available_ratio;
-    perf_counter_wrapper _counter_max_available_ratio;
-
     friend class replica_test;
     friend class replica_stub;
     friend class mock_replica_stub;
diff --git a/src/common/test/CMakeLists.txt b/src/common/test/CMakeLists.txt
index 78d94000c..74a9cdf3e 100644
--- a/src/common/test/CMakeLists.txt
+++ b/src/common/test/CMakeLists.txt
@@ -27,6 +27,7 @@ set(MY_PROJ_NAME dsn_replication_common_test)
 set(MY_SRC_SEARCH_MODE "GLOB")
 
 set(MY_PROJ_LIBS
+        dsn_http
         dsn_replication_common
         dsn_runtime
         gtest
diff --git a/src/meta/test/misc/misc.cpp b/src/meta/test/misc/misc.cpp
index 5733dd651..36a20806d 100644
--- a/src/meta/test/misc/misc.cpp
+++ b/src/meta/test/misc/misc.cpp
@@ -203,7 +203,7 @@ void generate_node_fs_manager(const app_mapper &apps,
     for (const auto &kv : nodes) {
         const node_state &ns = kv.second;
         if (nfm.find(ns.addr()) == nfm.end()) {
-            nfm.emplace(ns.addr(), std::make_shared<fs_manager>(true));
+            nfm.emplace(ns.addr(), std::make_shared<fs_manager>());
         }
         fs_manager &manager = *(nfm.find(ns.addr())->second);
         manager.initialize(data_dirs, tags, true);
diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp
index d5cf1f0bc..41b7e46a8 100644
--- a/src/replica/replica_stub.cpp
+++ b/src/replica/replica_stub.cpp
@@ -198,7 +198,6 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/,
       _mem_release_max_reserved_mem_percentage(10),
       _max_concurrent_bulk_load_downloading_count(5),
       _learn_app_concurrent_count(0),
-      _fs_manager(false),
       _bulk_load_downloading_count(0),
       _manual_emergency_checkpointing_count(0),
       _is_running(false)
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index e69268006..a230aa1f9 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -165,6 +165,7 @@ class error_code;
     _##name(METRIC_##name.instantiate(entity##_metric_entity(), ##__VA_ARGS__))
 #define METRIC_VAR_INIT_replica(name, ...) METRIC_VAR_INIT(name, replica, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_server(name, ...) METRIC_VAR_INIT(name, server, ##__VA_ARGS__)
+#define METRIC_VAR_INIT_disk(name, ...) METRIC_VAR_INIT(name, disk, ##__VA_ARGS__)
 
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
@@ -194,6 +195,11 @@ class error_code;
 
 #define METRIC_VAR_AUTO_LATENCY_DURATION_NS(name) __##name##_auto_latency.duration_ns()
 
+#define METRIC_DEFINE_SET_METHOD(name, value_type)                                                 \
+    void set_##name(value_type value) { METRIC_VAR_SET(name, value); }
+
+#define METRIC_CALL_SET_METHOD(obj, name, value) obj.set_##name(value)
+
 namespace dsn {
 class metric;                  // IWYU pragma: keep
 class metric_entity_prototype; // IWYU pragma: keep
@@ -614,6 +620,7 @@ enum class metric_unit : size_t
     kBytes,
     kMegaBytes,
     kCapacityUnits,
+    kPercent,
     kRequests,
     kSeeks,
     kPointLookups,


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


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

Posted by wa...@apache.org.
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 b01afb395259c513abd24929eca580513c243c68
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


[incubator-pegasus] 11/23: feat(new_metrics): migrate server-level metrics for nfs (#1421)

Posted by wa...@apache.org.
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 990da555bd43fdd2778c1318c9881a33b5762985
Author: Dan Wang <wa...@apache.org>
AuthorDate: Thu Mar 30 15:14:55 2023 +0800

    feat(new_metrics): migrate server-level metrics for nfs (#1421)
    
    https://github.com/apache/incubator-pegasus/issues/1329
    
    Migrate server-level metrics to new framework for both server/client sides
    of nfs, including the data size in bytes that are requested by client or read
    from local file in server, the number of nfs copy requests that fail for client
    or server, the data size in bytes that are written to local file in client, the
    number of failed writes to local file in client.
    
    The old type in perf counters of all these metrics are volatile counter, while
    all of them become accumulated counter for new metrics.
---
 src/nfs/nfs_client_impl.cpp         | 56 ++++++++++++++++++++-----------------
 src/nfs/nfs_client_impl.h           | 10 +++----
 src/nfs/nfs_server_impl.cpp         | 34 ++++++++++++----------
 src/nfs/nfs_server_impl.h           |  6 ++--
 src/nfs/test/CMakeLists.txt         |  2 +-
 src/server/pegasus_server_write.cpp |  1 -
 6 files changed, 60 insertions(+), 49 deletions(-)

diff --git a/src/nfs/nfs_client_impl.cpp b/src/nfs/nfs_client_impl.cpp
index c0ced7b8c..8ddadc46c 100644
--- a/src/nfs/nfs_client_impl.cpp
+++ b/src/nfs/nfs_client_impl.cpp
@@ -32,7 +32,6 @@
 
 #include "nfs/nfs_code_definition.h"
 #include "nfs/nfs_node.h"
-#include "perf_counter/perf_counter.h"
 #include "utils/blob.h"
 #include "utils/command_manager.h"
 #include "utils/filesystem.h"
@@ -40,8 +39,30 @@
 #include "utils/fmt_logging.h"
 #include "utils/ports.h"
 #include "utils/string_conv.h"
+#include "utils/string_view.h"
 #include "utils/token_buckets.h"
 
+METRIC_DEFINE_counter(server,
+                      nfs_client_copy_bytes,
+                      dsn::metric_unit::kBytes,
+                      "The accumulated data size in bytes requested by client during nfs copy");
+
+METRIC_DEFINE_counter(server,
+                      nfs_client_failed_copy_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of failed nfs copy requests (requested by client)");
+
+METRIC_DEFINE_counter(
+    server,
+    nfs_client_write_bytes,
+    dsn::metric_unit::kBytes,
+    "The accumulated data size in bytes that are written to local file in client");
+
+METRIC_DEFINE_counter(server,
+                      nfs_client_failed_writes,
+                      dsn::metric_unit::kWrites,
+                      "The number of failed writes to local file in client");
+
 namespace dsn {
 namespace service {
 static uint32_t current_max_copy_rate_megabytes = 0;
@@ -98,27 +119,12 @@ nfs_client_impl::nfs_client_impl()
       _concurrent_local_write_count(0),
       _buffered_local_write_count(0),
       _copy_requests_low(FLAGS_max_file_copy_request_count_per_file),
-      _high_priority_remaining_time(FLAGS_high_priority_speed_rate)
+      _high_priority_remaining_time(FLAGS_high_priority_speed_rate),
+      METRIC_VAR_INIT_server(nfs_client_copy_bytes),
+      METRIC_VAR_INIT_server(nfs_client_failed_copy_requests),
+      METRIC_VAR_INIT_server(nfs_client_write_bytes),
+      METRIC_VAR_INIT_server(nfs_client_failed_writes)
 {
-    _recent_copy_data_size.init_app_counter("eon.nfs_client",
-                                            "recent_copy_data_size",
-                                            COUNTER_TYPE_VOLATILE_NUMBER,
-                                            "nfs client copy data size in the recent period");
-    _recent_copy_fail_count.init_app_counter(
-        "eon.nfs_client",
-        "recent_copy_fail_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "nfs client copy fail count count in the recent period");
-    _recent_write_data_size.init_app_counter("eon.nfs_client",
-                                             "recent_write_data_size",
-                                             COUNTER_TYPE_VOLATILE_NUMBER,
-                                             "nfs client write data size in the recent period");
-    _recent_write_fail_count.init_app_counter(
-        "eon.nfs_client",
-        "recent_write_fail_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "nfs client write fail count count in the recent period");
-
     _copy_token_buckets = std::make_unique<utils::token_buckets>();
 
     register_cli_commands();
@@ -339,7 +345,7 @@ void nfs_client_impl::end_copy(::dsn::error_code err,
     }
 
     if (err != ::dsn::ERR_OK) {
-        _recent_copy_fail_count->increment();
+        METRIC_VAR_INCREMENT(nfs_client_failed_copy_requests);
 
         if (!fc->user_req->is_finished) {
             if (reqc->retry_count > 0) {
@@ -375,7 +381,7 @@ void nfs_client_impl::end_copy(::dsn::error_code err,
     }
 
     else {
-        _recent_copy_data_size->add(resp.size);
+        METRIC_VAR_INCREMENT_BY(nfs_client_copy_bytes, resp.size);
 
         reqc->response = resp;
         reqc->is_ready_for_write = true;
@@ -506,7 +512,7 @@ void nfs_client_impl::end_write(error_code err, size_t sz, const copy_request_ex
 
     bool completed = false;
     if (err != ERR_OK) {
-        _recent_write_fail_count->increment();
+        METRIC_VAR_INCREMENT(nfs_client_failed_writes);
 
         LOG_ERROR("[nfs_service] local write failed, dir = {}, file = {}, err = {}",
                   fc->user_req->file_size_req.dst_dir,
@@ -514,7 +520,7 @@ void nfs_client_impl::end_write(error_code err, size_t sz, const copy_request_ex
                   err);
         completed = true;
     } else {
-        _recent_write_data_size->add(sz);
+        METRIC_VAR_INCREMENT_BY(nfs_client_write_bytes, sz);
 
         file_wrapper_ptr temp_holder;
         zauto_lock l(fc->user_req->user_req_lock);
diff --git a/src/nfs/nfs_client_impl.h b/src/nfs/nfs_client_impl.h
index 88f70aad4..0c15fc8b3 100644
--- a/src/nfs/nfs_client_impl.h
+++ b/src/nfs/nfs_client_impl.h
@@ -41,7 +41,6 @@
 #include "aio/file_io.h"
 #include "nfs_code_definition.h"
 #include "nfs_types.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "runtime/rpc/rpc_address.h"
 #include "runtime/task/async_calls.h"
 #include "runtime/task/task.h"
@@ -50,6 +49,7 @@
 #include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/zlocks.h"
 
 namespace dsn {
@@ -311,10 +311,10 @@ private:
     zlock _local_writes_lock;
     std::deque<copy_request_ex_ptr> _local_writes;
 
-    perf_counter_wrapper _recent_copy_data_size;
-    perf_counter_wrapper _recent_copy_fail_count;
-    perf_counter_wrapper _recent_write_data_size;
-    perf_counter_wrapper _recent_write_fail_count;
+    METRIC_VAR_DECLARE_counter(nfs_client_copy_bytes);
+    METRIC_VAR_DECLARE_counter(nfs_client_failed_copy_requests);
+    METRIC_VAR_DECLARE_counter(nfs_client_write_bytes);
+    METRIC_VAR_DECLARE_counter(nfs_client_failed_writes);
 
     std::unique_ptr<command_deregister> _nfs_max_copy_rate_megabytes_cmd;
 
diff --git a/src/nfs/nfs_server_impl.cpp b/src/nfs/nfs_server_impl.cpp
index cadacba1a..25632d4f9 100644
--- a/src/nfs/nfs_server_impl.cpp
+++ b/src/nfs/nfs_server_impl.cpp
@@ -35,17 +35,30 @@
 #include <vector>
 
 #include "nfs/nfs_code_definition.h"
-#include "perf_counter/perf_counter.h"
 #include "runtime/api_layer1.h"
 #include "runtime/task/async_calls.h"
 #include "utils/TokenBucket.h"
+#include "utils/autoref_ptr.h"
 #include "utils/filesystem.h"
 #include "utils/flags.h"
 #include "utils/ports.h"
 #include "utils/safe_strerror_posix.h"
 #include "utils/string_conv.h"
+#include "utils/string_view.h"
 #include "utils/utils.h"
 
+METRIC_DEFINE_counter(
+    server,
+    nfs_server_copy_bytes,
+    dsn::metric_unit::kBytes,
+    "The accumulated data size in bytes that are read from local file in server during nfs copy");
+
+METRIC_DEFINE_counter(
+    server,
+    nfs_server_failed_copy_requests,
+    dsn::metric_unit::kRequests,
+    "The number of nfs copy requests (received by server) that fail to read local file in server");
+
 namespace dsn {
 class disk_file;
 
@@ -61,7 +74,10 @@ DSN_TAG_VARIABLE(max_send_rate_megabytes_per_disk, FT_MUTABLE);
 DSN_DECLARE_int32(file_close_timer_interval_ms_on_server);
 DSN_DECLARE_int32(file_close_expire_time_ms);
 
-nfs_service_impl::nfs_service_impl() : ::dsn::serverlet<nfs_service_impl>("nfs")
+nfs_service_impl::nfs_service_impl()
+    : ::dsn::serverlet<nfs_service_impl>("nfs"),
+      METRIC_VAR_INIT_server(nfs_server_copy_bytes),
+      METRIC_VAR_INIT_server(nfs_server_failed_copy_requests)
 {
     _file_close_timer = ::dsn::tasking::enqueue_timer(
         LPC_NFS_FILE_CLOSE_TIMER,
@@ -69,16 +85,6 @@ nfs_service_impl::nfs_service_impl() : ::dsn::serverlet<nfs_service_impl>("nfs")
         [this] { close_file(); },
         std::chrono::milliseconds(FLAGS_file_close_timer_interval_ms_on_server));
 
-    _recent_copy_data_size.init_app_counter("eon.nfs_server",
-                                            "recent_copy_data_size",
-                                            COUNTER_TYPE_VOLATILE_NUMBER,
-                                            "nfs server copy data size in the recent period");
-    _recent_copy_fail_count.init_app_counter(
-        "eon.nfs_server",
-        "recent_copy_fail_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "nfs server copy fail count count in the recent period");
-
     _send_token_buckets = std::make_unique<dsn::utils::token_buckets>();
     register_cli_commands();
 }
@@ -161,9 +167,9 @@ void nfs_service_impl::internal_read_callback(error_code err, size_t sz, callbac
 
     if (err != ERR_OK) {
         LOG_ERROR("[nfs_service] read file {} failed, err = {}", cp.file_path, err);
-        _recent_copy_fail_count->increment();
+        METRIC_VAR_INCREMENT(nfs_server_failed_copy_requests);
     } else {
-        _recent_copy_data_size->add(sz);
+        METRIC_VAR_INCREMENT_BY(nfs_server_copy_bytes, sz);
     }
 
     ::dsn::service::copy_response resp;
diff --git a/src/nfs/nfs_server_impl.h b/src/nfs/nfs_server_impl.h
index 9ba113404..4c07a4996 100644
--- a/src/nfs/nfs_server_impl.h
+++ b/src/nfs/nfs_server_impl.h
@@ -35,7 +35,6 @@
 #include "aio/file_io.h"
 #include "nfs_code_definition.h"
 #include "nfs_types.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "runtime/serverlet.h"
 #include "runtime/task/task.h"
 #include "runtime/task/task_tracker.h"
@@ -43,6 +42,7 @@
 #include "utils/command_manager.h"
 #include "utils/error_code.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/token_buckets.h"
 #include "utils/zlocks.h"
 
@@ -137,8 +137,8 @@ private:
     std::unique_ptr<dsn::utils::token_buckets>
         _send_token_buckets; // rate limiter of send to remote
 
-    perf_counter_wrapper _recent_copy_data_size;
-    perf_counter_wrapper _recent_copy_fail_count;
+    METRIC_VAR_DECLARE_counter(nfs_server_copy_bytes);
+    METRIC_VAR_DECLARE_counter(nfs_server_failed_copy_requests);
 
     std::unique_ptr<command_deregister> _nfs_max_send_rate_megabytes_cmd;
 
diff --git a/src/nfs/test/CMakeLists.txt b/src/nfs/test/CMakeLists.txt
index 735bb29bd..8dcf2cef7 100644
--- a/src/nfs/test/CMakeLists.txt
+++ b/src/nfs/test/CMakeLists.txt
@@ -33,7 +33,7 @@ set(MY_PROJ_SRC "")
 # "GLOB" for non-recursive search
 set(MY_SRC_SEARCH_MODE "GLOB")
 
-set(MY_PROJ_LIBS dsn_nfs dsn_runtime gtest dsn_aio)
+set(MY_PROJ_LIBS dsn_nfs dsn_runtime gtest dsn_aio dsn_http)
 
 set(MY_BOOST_LIBS Boost::system Boost::filesystem Boost::regex)
 
diff --git a/src/server/pegasus_server_write.cpp b/src/server/pegasus_server_write.cpp
index 9f8db6ca0..e0669fc99 100644
--- a/src/server/pegasus_server_write.cpp
+++ b/src/server/pegasus_server_write.cpp
@@ -18,7 +18,6 @@
  */
 
 #include <rocksdb/status.h>
-#include <stdio.h>
 #include <thrift/transport/TTransportException.h>
 #include <algorithm>
 #include <utility>


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


[incubator-pegasus] 06/23: feat(new_metrics): migrate replica-level metrics for replica class (#1397)

Posted by wa...@apache.org.
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 de1b95f40a91f6d45ddd22826791fef997531447
Author: Dan Wang <wa...@apache.org>
AuthorDate: Wed Mar 22 12:32:24 2023 +0800

    feat(new_metrics): migrate replica-level metrics for replica class (#1397)
    
    https://github.com/apache/incubator-pegasus/issues/1342
    
    Migrate replica-level metrics in `replica` class to new framework, including the size of
    private log, the number of backup requests, the number of read/write/backup requests
    that are delayed/rejected due to throttling, the number of read/write requests that are
    rejected due to splitting, the number of write requests that are rejected due to bulk load
    ingestion, the number of non-idempotent write requests that are rejected due to duplication.
    
    Note that the old metrics for the number of backup requests, the number of delayed/rejected
    backup requests due to throttling and the number of rejected non-idempotent write requests
    are table-level, which have been changed to replica-level in new metrics, since they could
    be aggregated to table-level if necessary.
---
 src/perf_counter/test/perf_counter_test.cpp  |   1 +
 src/replica/replica.cpp                      | 133 ++++++++++++++++-----------
 src/replica/replica.h                        |  38 ++++----
 src/replica/replica_2pc.cpp                  |   7 +-
 src/replica/replica_base.cpp                 |   2 +
 src/replica/replica_base.h                   |   3 +
 src/replica/replica_chkpt.cpp                |   8 +-
 src/replica/replica_throttle.cpp             |  11 +--
 src/replica/test/mutation_log_test.cpp       |   1 +
 src/replica/test/replica_test.cpp            |  14 +--
 src/server/capacity_unit_calculator.cpp      |   7 +-
 src/server/capacity_unit_calculator.h        |   3 +-
 src/server/pegasus_server_impl.cpp           |   1 +
 src/server/pegasus_server_impl.h             |   5 +-
 src/server/pegasus_server_impl_init.cpp      |   9 +-
 src/server/pegasus_server_write.cpp          |   2 +
 src/server/pegasus_server_write.h            |   1 +
 src/server/pegasus_write_service.cpp         |  11 ++-
 src/server/pegasus_write_service.h           |   3 +-
 src/server/rocksdb_wrapper.cpp               |   3 +-
 src/server/rocksdb_wrapper.h                 |   5 +-
 src/server/test/pegasus_server_impl_test.cpp |   4 +-
 src/utils/clock.cpp                          |   1 +
 23 files changed, 154 insertions(+), 119 deletions(-)

diff --git a/src/perf_counter/test/perf_counter_test.cpp b/src/perf_counter/test/perf_counter_test.cpp
index f83679bd4..0758ef807 100644
--- a/src/perf_counter/test/perf_counter_test.cpp
+++ b/src/perf_counter/test/perf_counter_test.cpp
@@ -38,6 +38,7 @@
 // IWYU pragma: no_include <gtest/gtest-test-part.h>
 #include <gtest/gtest.h>
 #include <stdlib.h>
+#include <algorithm>
 #include <chrono>
 #include <functional>
 #include <memory>
diff --git a/src/replica/replica.cpp b/src/replica/replica.cpp
index bcaa762df..8b8f22470 100644
--- a/src/replica/replica.cpp
+++ b/src/replica/replica.cpp
@@ -48,6 +48,7 @@
 #include "mutation.h"
 #include "mutation_log.h"
 #include "perf_counter/perf_counter.h"
+#include "perf_counter/perf_counter_wrapper.h"
 #include "perf_counter/perf_counters.h"
 #include "replica/prepare_list.h"
 #include "replica/replica_context.h"
@@ -67,6 +68,66 @@
 #include "utils/rand.h"
 #include "utils/string_view.h"
 
+METRIC_DEFINE_gauge_int64(replica,
+                          private_log_size_mb,
+                          dsn::metric_unit::kMegaBytes,
+                          "The size of private log in MB");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_delayed_write_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of delayed write requests by throttling");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_rejected_write_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected write requests by throttling");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_delayed_read_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of delayed read requests by throttling");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_rejected_read_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected read requests by throttling");
+
+METRIC_DEFINE_counter(replica,
+                      backup_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of backup requests");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_delayed_backup_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of delayed backup requests by throttling");
+
+METRIC_DEFINE_counter(replica,
+                      throttling_rejected_backup_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected backup requests by throttling");
+
+METRIC_DEFINE_counter(replica,
+                      splitting_rejected_write_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected write requests by splitting");
+
+METRIC_DEFINE_counter(replica,
+                      splitting_rejected_read_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected read requests by splitting");
+
+METRIC_DEFINE_counter(replica,
+                      bulk_load_ingestion_rejected_write_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected write requests by bulk load ingestion");
+
+METRIC_DEFINE_counter(replica,
+                      dup_rejected_non_idempotent_write_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of rejected non-idempotent write requests by duplication");
+
 namespace dsn {
 namespace replication {
 
@@ -119,7 +180,19 @@ replica::replica(replica_stub *stub,
       // todo(jiashuo1): app.duplicating need rename
       _is_duplication_master(app.duplicating),
       _is_duplication_follower(is_duplication_follower),
-      _backup_mgr(new replica_backup_manager(this))
+      _backup_mgr(new replica_backup_manager(this)),
+      METRIC_VAR_INIT_replica(private_log_size_mb),
+      METRIC_VAR_INIT_replica(throttling_delayed_write_requests),
+      METRIC_VAR_INIT_replica(throttling_rejected_write_requests),
+      METRIC_VAR_INIT_replica(throttling_delayed_read_requests),
+      METRIC_VAR_INIT_replica(throttling_rejected_read_requests),
+      METRIC_VAR_INIT_replica(backup_requests),
+      METRIC_VAR_INIT_replica(throttling_delayed_backup_requests),
+      METRIC_VAR_INIT_replica(throttling_rejected_backup_requests),
+      METRIC_VAR_INIT_replica(splitting_rejected_write_requests),
+      METRIC_VAR_INIT_replica(splitting_rejected_read_requests),
+      METRIC_VAR_INIT_replica(bulk_load_ingestion_rejected_write_requests),
+      METRIC_VAR_INIT_replica(dup_rejected_non_idempotent_write_requests)
 {
     CHECK(!_app_info.app_type.empty(), "");
     CHECK_NOTNULL(stub, "");
@@ -133,59 +206,9 @@ replica::replica(replica_stub *stub,
     _disk_migrator = std::make_unique<replica_disk_migrator>(this);
     _replica_follower = std::make_unique<replica_follower>(this);
 
-    std::string counter_str = fmt::format("private.log.size(MB)@{}", gpid);
-    _counter_private_log_size.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.write.throttling.delay.count@{}", gpid);
-    _counter_recent_write_throttling_delay_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.write.throttling.reject.count@{}", gpid);
-    _counter_recent_write_throttling_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.read.throttling.delay.count@{}", gpid);
-    _counter_recent_read_throttling_delay_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.read.throttling.reject.count@{}", gpid);
-    _counter_recent_read_throttling_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str =
-        fmt::format("recent.backup.request.throttling.delay.count@{}", _app_info.app_name);
-    _counter_recent_backup_request_throttling_delay_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str =
-        fmt::format("recent.backup.request.throttling.reject.count@{}", _app_info.app_name);
-    _counter_recent_backup_request_throttling_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("dup.disabled_non_idempotent_write_count@{}", _app_info.app_name);
-    _counter_dup_disabled_non_idempotent_write_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.read.splitting.reject.count@{}", gpid);
-    _counter_recent_read_splitting_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.write.splitting.reject.count@{}", gpid);
-    _counter_recent_write_splitting_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
-    counter_str = fmt::format("recent.write.bulk.load.ingestion.reject.count@{}", gpid);
-    _counter_recent_write_bulk_load_ingestion_reject_count.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_VOLATILE_NUMBER, counter_str.c_str());
-
     // init table level latency perf counters
     init_table_level_latency_counters();
 
-    counter_str = fmt::format("backup_request_qps@{}", _app_info.app_name);
-    _counter_backup_request_qps.init_app_counter(
-        "eon.replica", counter_str.c_str(), COUNTER_TYPE_RATE, counter_str.c_str());
-
     if (need_restore) {
         // add an extra env for restore
         _extra_envs.insert(
@@ -262,7 +285,7 @@ void replica::on_client_read(dsn::message_ex *request, bool ignore_throttling)
         return;
     }
 
-    CHECK_REQUEST_IF_SPLITTING(read)
+    CHECK_REQUEST_IF_SPLITTING(read);
 
     if (status() == partition_status::PS_INACTIVE ||
         status() == partition_status::PS_POTENTIAL_SECONDARY) {
@@ -295,7 +318,7 @@ void replica::on_client_read(dsn::message_ex *request, bool ignore_throttling)
         if (!ignore_throttling && throttle_backup_request(request)) {
             return;
         }
-        _counter_backup_request_qps->increment();
+        METRIC_VAR_INCREMENT(backup_requests);
     }
 
     uint64_t start_time_ns = dsn_now_ns();
@@ -526,8 +549,6 @@ void replica::close()
         _disk_migrator.reset();
     }
 
-    _counter_private_log_size.clear();
-
     // duplication_impl may have ongoing tasks.
     // release it before release replica.
     _duplication_mgr.reset();
@@ -614,5 +635,7 @@ bool replica::access_controller_allowed(message_ex *msg, const ranger::access_ty
     return !_access_controller->is_enable_ranger_acl() || _access_controller->allowed(msg, ac_type);
 }
 
+int64_t replica::get_backup_request_count() const { return METRIC_VAR_VALUE(backup_requests); }
+
 } // namespace replication
 } // namespace dsn
diff --git a/src/replica/replica.h b/src/replica/replica.h
index 2e8108371..104854e20 100644
--- a/src/replica/replica.h
+++ b/src/replica/replica.h
@@ -58,7 +58,6 @@
 #include "metadata_types.h"
 #include "mutation.h"
 #include "mutation_log.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "prepare_list.h"
 #include "replica/backup/cold_backup_context.h"
 #include "replica/replica_base.h"
@@ -72,6 +71,7 @@
 #include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/flags.h"
+#include "utils/metrics.h"
 #include "utils/thread_access_checker.h"
 #include "utils/throttling_controller.h"
 #include "utils/uniq_timestamp_us.h"
@@ -121,10 +121,13 @@ class test_checker;
 }
 
 #define CHECK_REQUEST_IF_SPLITTING(op_type)                                                        \
-    if (_validate_partition_hash) {                                                                \
+    do {                                                                                           \
+        if (!_validate_partition_hash) {                                                           \
+            break;                                                                                 \
+        }                                                                                          \
         if (_split_mgr->should_reject_request()) {                                                 \
             response_client_##op_type(request, ERR_SPLITTING);                                     \
-            _counter_recent_##op_type##_splitting_reject_count->increment();                       \
+            METRIC_VAR_INCREMENT(splitting_rejected_##op_type##_requests);                         \
             return;                                                                                \
         }                                                                                          \
         if (!_split_mgr->check_partition_hash(                                                     \
@@ -132,7 +135,7 @@ class test_checker;
             response_client_##op_type(request, ERR_PARENT_PARTITION_MISUSED);                      \
             return;                                                                                \
         }                                                                                          \
-    }
+    } while (0)
 
 DSN_DECLARE_bool(reject_write_when_disk_insufficient);
 
@@ -530,6 +533,9 @@ private:
     // use Apache Ranger for replica access control
     bool access_controller_allowed(message_ex *msg, const ranger::access_type &ac_type) const;
 
+    // Currently only used for unit test to get the count of backup requests.
+    int64_t get_backup_request_count() const;
+
 private:
     friend class ::dsn::replication::test::test_checker;
     friend class ::dsn::replication::mutation_queue;
@@ -649,19 +655,19 @@ private:
     std::unique_ptr<replica_follower> _replica_follower;
 
     // perf counters
-    perf_counter_wrapper _counter_private_log_size;
-    perf_counter_wrapper _counter_recent_write_throttling_delay_count;
-    perf_counter_wrapper _counter_recent_write_throttling_reject_count;
-    perf_counter_wrapper _counter_recent_read_throttling_delay_count;
-    perf_counter_wrapper _counter_recent_read_throttling_reject_count;
-    perf_counter_wrapper _counter_recent_backup_request_throttling_delay_count;
-    perf_counter_wrapper _counter_recent_backup_request_throttling_reject_count;
-    perf_counter_wrapper _counter_recent_write_splitting_reject_count;
-    perf_counter_wrapper _counter_recent_read_splitting_reject_count;
-    perf_counter_wrapper _counter_recent_write_bulk_load_ingestion_reject_count;
+    METRIC_VAR_DECLARE_gauge_int64(private_log_size_mb);
+    METRIC_VAR_DECLARE_counter(throttling_delayed_write_requests);
+    METRIC_VAR_DECLARE_counter(throttling_rejected_write_requests);
+    METRIC_VAR_DECLARE_counter(throttling_delayed_read_requests);
+    METRIC_VAR_DECLARE_counter(throttling_rejected_read_requests);
+    METRIC_VAR_DECLARE_counter(backup_requests);
+    METRIC_VAR_DECLARE_counter(throttling_delayed_backup_requests);
+    METRIC_VAR_DECLARE_counter(throttling_rejected_backup_requests);
+    METRIC_VAR_DECLARE_counter(splitting_rejected_write_requests);
+    METRIC_VAR_DECLARE_counter(splitting_rejected_read_requests);
+    METRIC_VAR_DECLARE_counter(bulk_load_ingestion_rejected_write_requests);
+    METRIC_VAR_DECLARE_counter(dup_rejected_non_idempotent_write_requests);
     std::vector<perf_counter *> _counters_table_level_latency;
-    perf_counter_wrapper _counter_dup_disabled_non_idempotent_write_count;
-    perf_counter_wrapper _counter_backup_request_qps;
 
     dsn::task_tracker _tracker;
     // the thread access checker
diff --git a/src/replica/replica_2pc.cpp b/src/replica/replica_2pc.cpp
index 0fd3b9851..9525466b2 100644
--- a/src/replica/replica_2pc.cpp
+++ b/src/replica/replica_2pc.cpp
@@ -73,6 +73,7 @@
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
 #include "utils/latency_tracer.h"
+#include "utils/metrics.h"
 #include "utils/ports.h"
 #include "utils/thread_access_checker.h"
 #include "utils/uniq_timestamp_us.h"
@@ -165,12 +166,12 @@ void replica::on_client_write(dsn::message_ex *request, bool ignore_throttling)
     if (is_duplication_master() && !spec->rpc_request_is_write_idempotent) {
         // Ignore non-idempotent write, because duplication provides no guarantee of atomicity to
         // make this write produce the same result on multiple clusters.
-        _counter_dup_disabled_non_idempotent_write_count->increment();
+        METRIC_VAR_INCREMENT(dup_rejected_non_idempotent_write_requests);
         response_client_write(request, ERR_OPERATION_DISABLED);
         return;
     }
 
-    CHECK_REQUEST_IF_SPLITTING(write)
+    CHECK_REQUEST_IF_SPLITTING(write);
 
     if (partition_status::PS_PRIMARY != status()) {
         response_client_write(request, ERR_INVALID_STATE);
@@ -186,7 +187,7 @@ void replica::on_client_write(dsn::message_ex *request, bool ignore_throttling)
     if (_is_bulk_load_ingestion) {
         if (request->rpc_code() != dsn::apps::RPC_RRDB_RRDB_BULK_LOAD) {
             // reject write requests during ingestion
-            _counter_recent_write_bulk_load_ingestion_reject_count->increment();
+            METRIC_VAR_INCREMENT(bulk_load_ingestion_rejected_write_requests);
             response_client_write(request, ERR_OPERATION_DISABLED);
         } else {
             response_client_write(request, ERR_NO_NEED_OPERATE);
diff --git a/src/replica/replica_base.cpp b/src/replica/replica_base.cpp
index 11e08ae05..3168ad651 100644
--- a/src/replica/replica_base.cpp
+++ b/src/replica/replica_base.cpp
@@ -18,6 +18,8 @@
 #include "replica_base.h"
 
 #include <fmt/core.h>
+#include <fmt/ostream.h>
+#include <iosfwd>
 
 METRIC_DEFINE_entity(replica);
 
diff --git a/src/replica/replica_base.h b/src/replica/replica_base.h
index 7c5b7747e..ccb39b914 100644
--- a/src/replica/replica_base.h
+++ b/src/replica/replica_base.h
@@ -26,7 +26,10 @@
 
 #pragma once
 
+#include <string>
+
 #include "common/gpid.h"
+#include "utils/fmt_logging.h"
 #include "utils/metrics.h"
 #include "utils/string_view.h"
 
diff --git a/src/replica/replica_chkpt.cpp b/src/replica/replica_chkpt.cpp
index b24f11157..5985e5d8d 100644
--- a/src/replica/replica_chkpt.cpp
+++ b/src/replica/replica_chkpt.cpp
@@ -70,6 +70,7 @@
 #include "utils/filesystem.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/thread_access_checker.h"
 
 namespace dsn {
@@ -168,9 +169,10 @@ void replica::on_checkpoint_timer()
                                  valid_start_offset,
                                  (int64_t)FLAGS_log_private_reserve_max_size_mb * 1024 * 1024,
                                  (int64_t)FLAGS_log_private_reserve_max_time_seconds);
-                             if (status() == partition_status::PS_PRIMARY)
-                                 _counter_private_log_size->set(_private_log->total_size() /
-                                                                1000000);
+                             if (status() == partition_status::PS_PRIMARY) {
+                                 METRIC_VAR_SET(private_log_size_mb,
+                                                _private_log->total_size() >> 20);
+                             }
                          });
     }
 }
diff --git a/src/replica/replica_throttle.cpp b/src/replica/replica_throttle.cpp
index 6a1a294b9..ed6bd1cce 100644
--- a/src/replica/replica_throttle.cpp
+++ b/src/replica/replica_throttle.cpp
@@ -25,14 +25,13 @@
 #include "common/replica_envs.h"
 #include "common/replication.codes.h"
 #include "dsn.layer2_types.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica.h"
 #include "runtime/rpc/rpc_message.h"
 #include "runtime/task/async_calls.h"
 #include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/throttling_controller.h"
 
 namespace dsn {
@@ -51,7 +50,7 @@ namespace replication {
                     [ this, req = message_ptr(request) ]() { on_client_##op_type(req, true); },    \
                     get_gpid().thread_hash(),                                                      \
                     std::chrono::milliseconds(delay_ms));                                          \
-                _counter_recent_##op_type##_throttling_delay_count->increment();                   \
+                METRIC_VAR_INCREMENT(throttling_delayed_##op_type##_requests);                     \
             } else { /** type == throttling_controller::REJECT **/                                 \
                 if (delay_ms > 0) {                                                                \
                     tasking::enqueue(LPC_##op_type##_THROTTLING_DELAY,                             \
@@ -64,7 +63,7 @@ namespace replication {
                 } else {                                                                           \
                     response_client_##op_type(request, ERR_BUSY);                                  \
                 }                                                                                  \
-                _counter_recent_##op_type##_throttling_reject_count->increment();                  \
+                METRIC_VAR_INCREMENT(throttling_rejected_##op_type##_requests);                    \
             }                                                                                      \
             return true;                                                                           \
         }                                                                                          \
@@ -95,9 +94,9 @@ bool replica::throttle_backup_request(message_ex *request)
                              [ this, req = message_ptr(request) ]() { on_client_read(req, true); },
                              get_gpid().thread_hash(),
                              std::chrono::milliseconds(delay_ms));
-            _counter_recent_backup_request_throttling_delay_count->increment();
+            METRIC_VAR_INCREMENT(throttling_delayed_backup_requests);
         } else { /** type == throttling_controller::REJECT **/
-            _counter_recent_backup_request_throttling_reject_count->increment();
+            METRIC_VAR_INCREMENT(throttling_rejected_backup_requests);
         }
         return true;
     }
diff --git a/src/replica/test/mutation_log_test.cpp b/src/replica/test/mutation_log_test.cpp
index 2b7ce0bab..44e8f7ac7 100644
--- a/src/replica/test/mutation_log_test.cpp
+++ b/src/replica/test/mutation_log_test.cpp
@@ -26,6 +26,7 @@
 
 #include "replica/mutation_log.h"
 
+#include <fcntl.h>
 // IWYU pragma: no_include <gtest/gtest-message.h>
 // IWYU pragma: no_include <gtest/gtest-test-part.h>
 #include <gtest/gtest.h>
diff --git a/src/replica/test/replica_test.cpp b/src/replica/test/replica_test.cpp
index 8e09a8e03..fbb872715 100644
--- a/src/replica/test/replica_test.cpp
+++ b/src/replica/test/replica_test.cpp
@@ -19,7 +19,6 @@
 // IWYU pragma: no_include <gtest/gtest-test-part.h>
 #include <gtest/gtest.h>
 #include <stdint.h>
-#include <unistd.h>
 #include <iostream>
 #include <map>
 #include <memory>
@@ -96,10 +95,7 @@ public:
         return stub->_counter_recent_write_size_exceed_threshold_count->get_value();
     }
 
-    int get_table_level_backup_request_qps()
-    {
-        return _mock_replica->_counter_backup_request_qps->get_integer_value();
-    }
+    int64_t get_backup_request_count() const { return _mock_replica->get_backup_request_count(); }
 
     bool get_validate_partition_hash() const { return _mock_replica->_validate_partition_hash; }
 
@@ -280,7 +276,7 @@ TEST_F(replica_test, write_size_limited)
     ASSERT_EQ(get_write_size_exceed_threshold_count(), count);
 }
 
-TEST_F(replica_test, backup_request_qps)
+TEST_F(replica_test, backup_request_count)
 {
     // create backup request
     struct dsn::message_header header;
@@ -292,11 +288,7 @@ TEST_F(replica_test, backup_request_qps)
     backup_request->io_session = sim_net->create_client_session(rpc_address());
 
     _mock_replica->on_client_read(backup_request);
-
-    // We have to sleep >= 0.1s, or the value this perf-counter will be 0, according to the
-    // implementation of perf-counter which type is COUNTER_TYPE_RATE.
-    usleep(1e5);
-    ASSERT_GT(get_table_level_backup_request_qps(), 0);
+    ASSERT_EQ(get_backup_request_count(), 1);
 }
 
 TEST_F(replica_test, query_data_version_test)
diff --git a/src/server/capacity_unit_calculator.cpp b/src/server/capacity_unit_calculator.cpp
index 7a62b3d56..55b9d104c 100644
--- a/src/server/capacity_unit_calculator.cpp
+++ b/src/server/capacity_unit_calculator.cpp
@@ -20,19 +20,18 @@
 #include "capacity_unit_calculator.h"
 
 #include <rocksdb/status.h>
-#include <stdio.h>
 #include <sys/param.h>
 #include <cmath>
-#include <string>
+#include <cstdint>
 
-#include "common/gpid.h"
 #include "hotkey_collector.h"
-#include "perf_counter/perf_counter.h"
 #include "rrdb/rrdb_types.h"
 #include "runtime/rpc/rpc_message.h"
+#include "utils/autoref_ptr.h"
 #include "utils/blob.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
 #include "utils/token_bucket_throttling_controller.h"
 
 METRIC_DEFINE_counter(replica,
diff --git a/src/server/capacity_unit_calculator.h b/src/server/capacity_unit_calculator.h
index 6d30a07ef..d69e3394b 100644
--- a/src/server/capacity_unit_calculator.h
+++ b/src/server/capacity_unit_calculator.h
@@ -23,12 +23,13 @@
 #include <memory>
 #include <vector>
 
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica_base.h"
+#include "utils/metrics.h"
 
 namespace dsn {
 class blob;
 class message_ex;
+
 namespace apps {
 class full_data;
 class key_value;
diff --git a/src/server/pegasus_server_impl.cpp b/src/server/pegasus_server_impl.cpp
index 7d0dad595..c8d3ed601 100644
--- a/src/server/pegasus_server_impl.cpp
+++ b/src/server/pegasus_server_impl.cpp
@@ -57,6 +57,7 @@
 #include "pegasus_rpc_types.h"
 #include "pegasus_server_write.h"
 #include "perf_counter/perf_counter.h"
+#include "perf_counter/perf_counter_wrapper.h"
 #include "replica_admin_types.h"
 #include "rrdb/rrdb.code.definition.h"
 #include "rrdb/rrdb_types.h"
diff --git a/src/server/pegasus_server_impl.h b/src/server/pegasus_server_impl.h
index c107089af..6a7817017 100644
--- a/src/server/pegasus_server_impl.h
+++ b/src/server/pegasus_server_impl.h
@@ -42,13 +42,13 @@
 #include "pegasus_scan_context.h"
 #include "pegasus_utils.h"
 #include "pegasus_value_schema.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "range_read_limiter.h"
 #include "replica/replication_app_base.h"
 #include "runtime/task/task.h"
 #include "runtime/task/task_tracker.h"
 #include "utils/error_code.h"
 #include "utils/flags.h"
+#include "utils/metrics.h"
 #include "utils/rand.h"
 #include "utils/synchronize.h"
 
@@ -69,6 +69,9 @@ class WriteBufferManager;
 namespace dsn {
 class blob;
 class message_ex;
+class perf_counter_wrapper;
+class rpc_address;
+
 namespace replication {
 class detect_hotkey_request;
 class detect_hotkey_response;
diff --git a/src/server/pegasus_server_impl_init.cpp b/src/server/pegasus_server_impl_init.cpp
index 0840c094b..cd51f0575 100644
--- a/src/server/pegasus_server_impl_init.cpp
+++ b/src/server/pegasus_server_impl_init.cpp
@@ -17,7 +17,6 @@
  * under the License.
  */
 
-#include <fmt/core.h>
 #include <rocksdb/cache.h>
 #include <rocksdb/filter_policy.h>
 #include <rocksdb/options.h>
@@ -29,6 +28,7 @@
 #include <cstdint>
 #include <memory>
 #include <mutex>
+#include <set>
 #include <string>
 #include <unordered_map>
 #include <utility>
@@ -53,6 +53,8 @@
 #include "server/range_read_limiter.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
+#include "utils/string_view.h"
 #include "utils/strings.h"
 #include "utils/token_bucket_throttling_controller.h"
 
@@ -117,10 +119,7 @@ METRIC_DEFINE_counter(replica,
                       dsn::metric_unit::kRequests,
                       "The number of abnormal read requests");
 
-METRIC_DEFINE_counter(replica,
-                      throttling_rejected_read_requests,
-                      dsn::metric_unit::kRequests,
-                      "The number of rejected read requests by throttling");
+METRIC_DECLARE_counter(throttling_rejected_read_requests);
 
 METRIC_DEFINE_gauge_int64(replica,
                           rdb_total_sst_files,
diff --git a/src/server/pegasus_server_write.cpp b/src/server/pegasus_server_write.cpp
index e6cb5331d..9f8db6ca0 100644
--- a/src/server/pegasus_server_write.cpp
+++ b/src/server/pegasus_server_write.cpp
@@ -36,10 +36,12 @@
 #include "runtime/rpc/rpc_holder.h"
 #include "runtime/rpc/rpc_message.h"
 #include "server/pegasus_write_service.h"
+#include "utils/autoref_ptr.h"
 #include "utils/blob.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
 #include "utils/ports.h"
+#include "utils/string_view.h"
 
 METRIC_DEFINE_counter(replica,
                       corrupt_writes,
diff --git a/src/server/pegasus_server_write.h b/src/server/pegasus_server_write.h
index d8a358164..2329ec586 100644
--- a/src/server/pegasus_server_write.h
+++ b/src/server/pegasus_server_write.h
@@ -30,6 +30,7 @@
 #include "replica/replica_base.h"
 #include "rrdb/rrdb_types.h"
 #include "runtime/task/task_code.h"
+#include "utils/metrics.h"
 
 namespace dsn {
 class blob;
diff --git a/src/server/pegasus_write_service.cpp b/src/server/pegasus_write_service.cpp
index 969e7d141..22c0fc1fe 100644
--- a/src/server/pegasus_write_service.cpp
+++ b/src/server/pegasus_write_service.cpp
@@ -20,21 +20,21 @@
 #include <fmt/core.h>
 #include <fmt/ostream.h>
 #include <rocksdb/status.h>
-#include <algorithm>
+#include <stddef.h>
+#include <functional>
 #include <iosfwd>
-#include <string>
+#include <set>
+#include <vector>
 
 #include "base/pegasus_rpc_types.h"
 #include "bulk_load_types.h"
 #include "capacity_unit_calculator.h"
 #include "common/duplication_common.h"
-#include "common/gpid.h"
 #include "common/replication.codes.h"
 #include "duplication_internal_types.h"
 #include "pegasus_value_schema.h"
 #include "pegasus_write_service.h"
 #include "pegasus_write_service_impl.h"
-#include "perf_counter/perf_counter.h"
 #include "rrdb/rrdb.code.definition.h"
 #include "rrdb/rrdb_types.h"
 #include "runtime/api_layer1.h"
@@ -42,10 +42,11 @@
 #include "runtime/task/async_calls.h"
 #include "runtime/task/task_code.h"
 #include "server/pegasus_server_impl.h"
-#include "utils/defer.h"
+#include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
 
 namespace dsn {
 class blob;
diff --git a/src/server/pegasus_write_service.h b/src/server/pegasus_write_service.h
index 9e79f9122..6c63c42d1 100644
--- a/src/server/pegasus_write_service.h
+++ b/src/server/pegasus_write_service.h
@@ -21,15 +21,16 @@
 
 #include <cstdint>
 #include <memory>
-#include <vector>
 
 #include "common//duplication_common.h"
 #include "common/common.h"
 #include "replica/replica_base.h"
 #include "utils/errors.h"
+#include "utils/metrics.h"
 
 namespace dsn {
 class blob;
+
 namespace apps {
 class check_and_mutate_request;
 class check_and_mutate_response;
diff --git a/src/server/rocksdb_wrapper.cpp b/src/server/rocksdb_wrapper.cpp
index 08fe8aeca..1465211a4 100644
--- a/src/server/rocksdb_wrapper.cpp
+++ b/src/server/rocksdb_wrapper.cpp
@@ -27,12 +27,11 @@
 #include "pegasus_key_schema.h"
 #include "pegasus_utils.h"
 #include "pegasus_write_service_impl.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "server/logging_utils.h"
 #include "server/meta_store.h"
 #include "server/pegasus_server_impl.h"
 #include "server/pegasus_write_service.h"
+#include "utils/autoref_ptr.h"
 #include "utils/blob.h"
 #include "utils/fail_point.h"
 #include "utils/flags.h"
diff --git a/src/server/rocksdb_wrapper.h b/src/server/rocksdb_wrapper.h
index fef30a629..e3eb600db 100644
--- a/src/server/rocksdb_wrapper.h
+++ b/src/server/rocksdb_wrapper.h
@@ -29,6 +29,7 @@
 
 #include "pegasus_value_schema.h"
 #include "replica/replica_base.h"
+#include "utils/metrics.h"
 #include "utils/string_view.h"
 
 namespace rocksdb {
@@ -36,10 +37,6 @@ class ColumnFamilyHandle;
 class DB;
 } // namespace rocksdb
 
-namespace dsn {
-class perf_counter_wrapper;
-} // namespace dsn
-
 namespace pegasus {
 
 namespace server {
diff --git a/src/server/test/pegasus_server_impl_test.cpp b/src/server/test/pegasus_server_impl_test.cpp
index 075193fef..880eeb403 100644
--- a/src/server/test/pegasus_server_impl_test.cpp
+++ b/src/server/test/pegasus_server_impl_test.cpp
@@ -33,15 +33,15 @@
 
 #include "pegasus_const.h"
 #include "pegasus_server_test_base.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "rrdb/rrdb.code.definition.h"
 #include "rrdb/rrdb_types.h"
 #include "runtime/serverlet.h"
 #include "server/pegasus_read_service.h"
+#include "utils/autoref_ptr.h"
 #include "utils/blob.h"
 #include "utils/error_code.h"
 #include "utils/filesystem.h"
+#include "utils/metrics.h"
 
 namespace pegasus {
 namespace server {
diff --git a/src/utils/clock.cpp b/src/utils/clock.cpp
index 84764e908..cb1fd400c 100644
--- a/src/utils/clock.cpp
+++ b/src/utils/clock.cpp
@@ -17,6 +17,7 @@
 
 #include "clock.h"
 
+#include "runtime/api_layer1.h"
 #include "time_utils.h"
 
 uint64_t dsn_now_ns() { return dsn::utils::clock::instance()->now_ns(); }


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


[incubator-pegasus] 16/23: feat(new_metrics): add backup-policy-level metric entity and migrate backup-policy-level metrics for meta_backup_service (#1438)

Posted by wa...@apache.org.
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 e83402be6b39c4b957c6d6593dbd8e481d0a2285
Author: Dan Wang <wa...@apache.org>
AuthorDate: Sat Apr 15 23:10:21 2023 +0800

    feat(new_metrics): add backup-policy-level metric entity and migrate backup-policy-level metrics for meta_backup_service (#1438)
    
    https://github.com/apache/incubator-pegasus/issues/1331
    
    In perf counters, there's only one metric for meta_backup_service, namely
    the recent backup duration for each policy, which means this metric is
    policy-level. Therefore policy-level entity would also be implemented in
    new metrics.
---
 src/meta/meta_backup_service.cpp | 48 +++++++++++++++++++++++++++++++---------
 src/meta/meta_backup_service.h   | 28 ++++++++++++++++++++---
 src/meta/test/backup_test.cpp    |  3 +--
 src/utils/metrics.h              |  1 +
 4 files changed, 65 insertions(+), 15 deletions(-)

diff --git a/src/meta/meta_backup_service.cpp b/src/meta/meta_backup_service.cpp
index db2afe1d9..cfdddbb53 100644
--- a/src/meta/meta_backup_service.cpp
+++ b/src/meta/meta_backup_service.cpp
@@ -35,7 +35,6 @@
 #include "meta/meta_state_service.h"
 #include "meta_backup_service.h"
 #include "meta_service.h"
-#include "perf_counter/perf_counter.h"
 #include "runtime/api_layer1.h"
 #include "runtime/rpc/rpc_address.h"
 #include "runtime/rpc/rpc_holder.h"
@@ -50,14 +49,47 @@
 #include "utils/chrono_literals.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
 #include "utils/time_utils.h"
 
+METRIC_DEFINE_entity(backup_policy);
+
+METRIC_DEFINE_gauge_int64(backup_policy,
+                          policy_recent_backup_duration_ms,
+                          dsn::metric_unit::kMilliSeconds,
+                          "The recent backup duration");
+
 namespace dsn {
 namespace replication {
 
 DSN_DECLARE_int32(cold_backup_checkpoint_reserve_minutes);
 DSN_DECLARE_int32(fd_lease_seconds);
 
+namespace {
+
+metric_entity_ptr instantiate_backup_policy_metric_entity(const std::string &policy_name)
+{
+    auto entity_id = fmt::format("backup_policy_{}", policy_name);
+
+    return METRIC_ENTITY_backup_policy.instantiate(entity_id, {{"policy_name", policy_name}});
+}
+
+} // anonymous namespace
+
+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)
+{
+}
+
+const metric_entity_ptr &backup_policy_metrics::backup_policy_metric_entity() const
+{
+    CHECK_NOTNULL(_backup_policy_metric_entity,
+                  "backup_policy metric entity should has been instantiated: "
+                  "uninitialized entity cannot be used to instantiate metric");
+    return _backup_policy_metric_entity;
+}
+
 // TODO: backup_service and policy_context should need two locks, its own _lock and server_state's
 // _lock this maybe lead to deadlock, should refactor this
 
@@ -830,12 +862,8 @@ void policy_context::start()
         continue_current_backup_unlocked();
     }
 
-    std::string counter_name = _policy.policy_name + ".recent.backup.duration(ms)";
-    _counter_policy_recent_backup_duration_ms.init_app_counter(
-        "eon.meta.policy",
-        counter_name.c_str(),
-        COUNTER_TYPE_NUMBER,
-        "policy recent backup duration time");
+    CHECK(!_policy.policy_name.empty(), "policy_name should has been initialized");
+    _metrics = std::make_unique<backup_policy_metrics>(_policy.policy_name);
 
     issue_gc_backup_info_task_unlocked();
     LOG_INFO("{}: start gc backup info task succeed", _policy.policy_name);
@@ -1011,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);
         }
     }
-    _counter_policy_recent_backup_duration_ms->set(last_backup_duration_time_ms);
+    METRIC_SET(*_metrics, policy_recent_backup_duration_ms, last_backup_duration_time_ms);
 }
 
 void policy_context::sync_remove_backup_info(const backup_info &info, dsn::task_ptr sync_callback)
@@ -1202,7 +1230,7 @@ error_code backup_service::sync_policies_from_remote_storage()
                         std::shared_ptr<policy_context> policy_ctx = _factory(this);
                         policy tpolicy;
                         dsn::json::json_forwarder<policy>::decode(value, tpolicy);
-                        policy_ctx->set_policy(std::move(tpolicy));
+                        policy_ctx->set_policy(tpolicy);
 
                         {
                             zauto_lock l(_lock);
@@ -1334,7 +1362,7 @@ void backup_service::add_backup_policy(dsn::message_ex *msg)
     p.start_time.parse_from(request.start_time);
     p.app_ids = app_ids;
     p.app_names = app_names;
-    policy_context_ptr->set_policy(std::move(p));
+    policy_context_ptr->set_policy(p);
     do_add_policy(msg, policy_context_ptr, response.hint_message);
 }
 
diff --git a/src/meta/meta_backup_service.h b/src/meta/meta_backup_service.h
index e6563dbcc..021b99552 100644
--- a/src/meta/meta_backup_service.h
+++ b/src/meta/meta_backup_service.h
@@ -36,16 +36,19 @@
 #include "common/json_helper.h"
 #include "common/replication_other_types.h"
 #include "meta_rpc_types.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "runtime/task/task.h"
 #include "runtime/task/task_tracker.h"
 #include "utils/api_utilities.h"
+#include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
+#include "utils/metrics.h"
+#include "utils/ports.h"
 #include "utils/zlocks.h"
 
 namespace dsn {
 class message_ex;
 class rpc_address;
+
 namespace dist {
 namespace block_service {
 class block_filesystem;
@@ -173,6 +176,23 @@ struct backup_start_time
     DEFINE_JSON_SERIALIZATION(hour, minute)
 };
 
+class backup_policy_metrics
+{
+public:
+    backup_policy_metrics() = default;
+    backup_policy_metrics(const std::string &policy_name);
+
+    const metric_entity_ptr &backup_policy_metric_entity() const;
+
+    METRIC_DEFINE_SET(policy_recent_backup_duration_ms, int64_t)
+
+private:
+    const metric_entity_ptr _backup_policy_metric_entity;
+    METRIC_VAR_DECLARE_gauge_int64(policy_recent_backup_duration_ms);
+
+    DISALLOW_COPY_AND_ASSIGN(backup_policy_metrics);
+};
+
 //
 // the backup process of meta server:
 //      1, write the app metadata to block filesystem
@@ -193,6 +213,7 @@ public:
     int32_t backup_history_count_to_keep;
     bool is_disable;
     backup_start_time start_time;
+
     policy()
         : app_ids(),
           backup_interval_seconds(0),
@@ -327,8 +348,9 @@ mock_private :
     backup_progress _progress;
     std::string _backup_sig; // policy_name@backup_id, used when print backup related log
 
-    perf_counter_wrapper _counter_policy_recent_backup_duration_ms;
-//clang-format on
+    std::unique_ptr<backup_policy_metrics> _metrics;
+
+    //clang-format on
     dsn::task_tracker _tracker;
 };
 
diff --git a/src/meta/test/backup_test.cpp b/src/meta/test/backup_test.cpp
index 35b88bb97..be6aaf666 100644
--- a/src/meta/test/backup_test.cpp
+++ b/src/meta/test/backup_test.cpp
@@ -29,7 +29,6 @@
 #include <set>
 #include <string>
 #include <thread>
-#include <type_traits>
 #include <utility>
 #include <vector>
 
@@ -272,7 +271,7 @@ protected:
         _policy.app_names[4] = "app4";
         _policy.app_names[6] = "app6";
         _mp._backup_service = _service->_backup_handler.get();
-        _mp.set_policy(policy(_policy));
+        _mp.set_policy(_policy);
 
         _service->_storage
             ->create_node(
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index 377bb58d7..bdc0ccd50 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -168,6 +168,7 @@ class error_code;
 #define METRIC_VAR_INIT_disk(name, ...) METRIC_VAR_INIT(name, disk, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_table(name, ...) METRIC_VAR_INIT(name, table, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_partition(name, ...) METRIC_VAR_INIT(name, partition, ##__VA_ARGS__)
+#define METRIC_VAR_INIT_backup_policy(name, ...) METRIC_VAR_INIT(name, backup_policy, ##__VA_ARGS__)
 
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \


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


[incubator-pegasus] 02/23: feat(new_metrics): migrate replica-level metrics for write service (#1351)

Posted by wa...@apache.org.
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 bcec273f8809520f763df7bc2e3a56a51049eac7
Author: Dan Wang <wa...@apache.org>
AuthorDate: Thu Feb 23 23:18:20 2023 +0800

    feat(new_metrics): migrate replica-level metrics for write service (#1351)
---
 .github/workflows/lint_and_test_cpp.yaml      |  12 +-
 run.sh                                        |   6 +-
 src/replica/replica_base.h                    |   9 +-
 src/server/pegasus_server_write.cpp           |  21 +-
 src/server/pegasus_server_write.h             |   3 +-
 src/server/pegasus_write_service.cpp          | 284 ++++++++++++++------------
 src/server/pegasus_write_service.h            |  47 +++--
 src/server/test/pegasus_server_write_test.cpp |   4 +-
 src/utils/metrics.h                           | 135 +++++++++++-
 src/utils/test/metrics_test.cpp               | 147 ++++++++++++-
 src/utils/time_utils.h                        |  28 ++-
 11 files changed, 513 insertions(+), 183 deletions(-)

diff --git a/.github/workflows/lint_and_test_cpp.yaml b/.github/workflows/lint_and_test_cpp.yaml
index 8b10be60f..032e632e3 100644
--- a/.github/workflows/lint_and_test_cpp.yaml
+++ b/.github/workflows/lint_and_test_cpp.yaml
@@ -172,7 +172,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
@@ -295,7 +299,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
diff --git a/run.sh b/run.sh
index cabf53402..0df98b040 100755
--- a/run.sh
+++ b/run.sh
@@ -342,7 +342,11 @@ function run_test()
       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
diff --git a/src/replica/replica_base.h b/src/replica/replica_base.h
index 88202d055..7c5b7747e 100644
--- a/src/replica/replica_base.h
+++ b/src/replica/replica_base.h
@@ -51,7 +51,14 @@ struct replica_base
 
     const char *log_prefix() const { return _name.c_str(); }
 
-    const metric_entity_ptr &replica_metric_entity() const { return _replica_metric_entity; }
+    const metric_entity_ptr &replica_metric_entity() const
+    {
+        CHECK_NOTNULL(_replica_metric_entity,
+                      "replica metric entity should has been instantiated: "
+                      "uninitialized entity cannot be used to instantiate "
+                      "metric");
+        return _replica_metric_entity;
+    }
 
 private:
     const gpid _gpid;
diff --git a/src/server/pegasus_server_write.cpp b/src/server/pegasus_server_write.cpp
index 0147050a6..e6cb5331d 100644
--- a/src/server/pegasus_server_write.cpp
+++ b/src/server/pegasus_server_write.cpp
@@ -31,7 +31,6 @@
 #include "pegasus_server_impl.h"
 #include "pegasus_server_write.h"
 #include "pegasus_utils.h"
-#include "perf_counter/perf_counter.h"
 #include "rrdb/rrdb.code.definition.h"
 #include "runtime/rpc/rpc_address.h"
 #include "runtime/rpc/rpc_holder.h"
@@ -42,20 +41,20 @@
 #include "utils/fmt_logging.h"
 #include "utils/ports.h"
 
+METRIC_DEFINE_counter(replica,
+                      corrupt_writes,
+                      dsn::metric_unit::kRequests,
+                      "The number of corrupt writes for each replica");
+
 namespace pegasus {
 namespace server {
 DSN_DECLARE_bool(rocksdb_verbose_log);
 
 pegasus_server_write::pegasus_server_write(pegasus_server_impl *server)
-    : replica_base(server), _write_svc(new pegasus_write_service(server))
+    : replica_base(server),
+      _write_svc(new pegasus_write_service(server)),
+      METRIC_VAR_INIT_replica(corrupt_writes)
 {
-    char name[256];
-    snprintf(name, 255, "recent_corrupt_write_count@%s", get_gpid().to_string());
-    _pfc_recent_corrupt_write_count.init_app_counter("app.pegasus",
-                                                     name,
-                                                     COUNTER_TYPE_VOLATILE_NUMBER,
-                                                     "statistic the recent corrupt write count");
-
     init_non_batch_write_handlers();
 }
 
@@ -81,7 +80,7 @@ int pegasus_server_write::on_batched_write_requests(dsn::message_ex **requests,
             return iter->second(requests[0]);
         }
     } catch (TTransportException &ex) {
-        _pfc_recent_corrupt_write_count->increment();
+        METRIC_VAR_INCREMENT(corrupt_writes);
         LOG_ERROR_PREFIX("pegasus not batch write handler failed, from = {}, exception = {}",
                          requests[0]->header->from_address.to_string(),
                          ex.what());
@@ -125,7 +124,7 @@ int pegasus_server_write::on_batched_writes(dsn::message_ex **requests, int coun
                     }
                 }
             } catch (TTransportException &ex) {
-                _pfc_recent_corrupt_write_count->increment();
+                METRIC_VAR_INCREMENT(corrupt_writes);
                 LOG_ERROR_PREFIX("pegasus batch writes handler failed, from = {}, exception = {}",
                                  requests[i]->header->from_address.to_string(),
                                  ex.what());
diff --git a/src/server/pegasus_server_write.h b/src/server/pegasus_server_write.h
index 6a002c299..d8a358164 100644
--- a/src/server/pegasus_server_write.h
+++ b/src/server/pegasus_server_write.h
@@ -27,7 +27,6 @@
 
 #include "base/pegasus_rpc_types.h"
 #include "pegasus_write_service.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica_base.h"
 #include "rrdb/rrdb_types.h"
 #include "runtime/task/task_code.h"
@@ -102,7 +101,7 @@ private:
     typedef std::map<dsn::task_code, std::function<int(dsn::message_ex *)>> non_batch_writes_map;
     non_batch_writes_map _non_batch_write_handlers;
 
-    ::dsn::perf_counter_wrapper _pfc_recent_corrupt_write_count;
+    METRIC_VAR_DECLARE_counter(corrupt_writes);
 };
 
 } // namespace server
diff --git a/src/server/pegasus_write_service.cpp b/src/server/pegasus_write_service.cpp
index 73f6cc8d6..4889329d9 100644
--- a/src/server/pegasus_write_service.cpp
+++ b/src/server/pegasus_write_service.cpp
@@ -52,6 +52,93 @@ class blob;
 class message_ex;
 } // namespace dsn
 
+METRIC_DEFINE_counter(replica,
+                      put_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of PUT requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      multi_put_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of MULTI_PUT requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      remove_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of REMOVE requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      multi_remove_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of MULTI_REMOVE requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      incr_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of INCR requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      check_and_set_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of CHECK_AND_SET requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      check_and_mutate_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of CHECK_AND_MUTATE requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               put_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of PUT requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               multi_put_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of MULTI_PUT requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               remove_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of REMOVE requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               multi_remove_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of MULTI_REMOVE requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               incr_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of INCR requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               check_and_set_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of CHECK_AND_SET requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               check_and_mutate_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of CHECK_AND_MUTATE requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      dup_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of DUPLICATE requests for each replica");
+
+METRIC_DEFINE_percentile_int64(
+    replica,
+    dup_time_lag_ms,
+    dsn::metric_unit::kMilliSeconds,
+    "the time lag (in ms) between master and slave in the duplication for each replica");
+
+METRIC_DEFINE_counter(
+    replica,
+    dup_lagging_writes,
+    dsn::metric_unit::kRequests,
+    "the number of lagging writes (time lag larger than `dup_lagging_write_threshold_ms`)");
+
 namespace pegasus {
 namespace server {
 
@@ -68,105 +155,33 @@ pegasus_write_service::pegasus_write_service(pegasus_server_impl *server)
       _server(server),
       _impl(new impl(server)),
       _batch_start_time(0),
-      _cu_calculator(server->_cu_calculator.get())
+      _cu_calculator(server->_cu_calculator.get()),
+      METRIC_VAR_INIT_replica(put_requests),
+      METRIC_VAR_INIT_replica(multi_put_requests),
+      METRIC_VAR_INIT_replica(remove_requests),
+      METRIC_VAR_INIT_replica(multi_remove_requests),
+      METRIC_VAR_INIT_replica(incr_requests),
+      METRIC_VAR_INIT_replica(check_and_set_requests),
+      METRIC_VAR_INIT_replica(check_and_mutate_requests),
+      METRIC_VAR_INIT_replica(put_latency_ns),
+      METRIC_VAR_INIT_replica(multi_put_latency_ns),
+      METRIC_VAR_INIT_replica(remove_latency_ns),
+      METRIC_VAR_INIT_replica(multi_remove_latency_ns),
+      METRIC_VAR_INIT_replica(incr_latency_ns),
+      METRIC_VAR_INIT_replica(check_and_set_latency_ns),
+      METRIC_VAR_INIT_replica(check_and_mutate_latency_ns),
+      METRIC_VAR_INIT_replica(dup_requests),
+      METRIC_VAR_INIT_replica(dup_time_lag_ms),
+      METRIC_VAR_INIT_replica(dup_lagging_writes),
+      _put_batch_size(0),
+      _remove_batch_size(0)
 {
-    std::string str_gpid = fmt::format("{}", server->get_gpid());
-
-    std::string name;
-
-    name = fmt::format("put_qps@{}", str_gpid);
-    _pfc_put_qps.init_app_counter(
-        "app.pegasus", name.c_str(), COUNTER_TYPE_RATE, "statistic the qps of PUT request");
-
-    name = fmt::format("multi_put_qps@{}", str_gpid);
-    _pfc_multi_put_qps.init_app_counter(
-        "app.pegasus", name.c_str(), COUNTER_TYPE_RATE, "statistic the qps of MULTI_PUT request");
-
-    name = fmt::format("remove_qps@{}", str_gpid);
-    _pfc_remove_qps.init_app_counter(
-        "app.pegasus", name.c_str(), COUNTER_TYPE_RATE, "statistic the qps of REMOVE request");
-
-    name = fmt::format("multi_remove_qps@{}", str_gpid);
-    _pfc_multi_remove_qps.init_app_counter("app.pegasus",
-                                           name.c_str(),
-                                           COUNTER_TYPE_RATE,
-                                           "statistic the qps of MULTI_REMOVE request");
-
-    name = fmt::format("incr_qps@{}", str_gpid);
-    _pfc_incr_qps.init_app_counter(
-        "app.pegasus", name.c_str(), COUNTER_TYPE_RATE, "statistic the qps of INCR request");
-
-    name = fmt::format("check_and_set_qps@{}", str_gpid);
-    _pfc_check_and_set_qps.init_app_counter("app.pegasus",
-                                            name.c_str(),
-                                            COUNTER_TYPE_RATE,
-                                            "statistic the qps of CHECK_AND_SET request");
-
-    name = fmt::format("check_and_mutate_qps@{}", str_gpid);
-    _pfc_check_and_mutate_qps.init_app_counter("app.pegasus",
-                                               name.c_str(),
-                                               COUNTER_TYPE_RATE,
-                                               "statistic the qps of CHECK_AND_MUTATE request");
-
-    name = fmt::format("put_latency@{}", str_gpid);
-    _pfc_put_latency.init_app_counter("app.pegasus",
-                                      name.c_str(),
-                                      COUNTER_TYPE_NUMBER_PERCENTILES,
-                                      "statistic the latency of PUT request");
-
-    name = fmt::format("multi_put_latency@{}", str_gpid);
-    _pfc_multi_put_latency.init_app_counter("app.pegasus",
-                                            name.c_str(),
-                                            COUNTER_TYPE_NUMBER_PERCENTILES,
-                                            "statistic the latency of MULTI_PUT request");
-
-    name = fmt::format("remove_latency@{}", str_gpid);
-    _pfc_remove_latency.init_app_counter("app.pegasus",
-                                         name.c_str(),
-                                         COUNTER_TYPE_NUMBER_PERCENTILES,
-                                         "statistic the latency of REMOVE request");
-
-    name = fmt::format("multi_remove_latency@{}", str_gpid);
-    _pfc_multi_remove_latency.init_app_counter("app.pegasus",
-                                               name.c_str(),
-                                               COUNTER_TYPE_NUMBER_PERCENTILES,
-                                               "statistic the latency of MULTI_REMOVE request");
-
-    name = fmt::format("incr_latency@{}", str_gpid);
-    _pfc_incr_latency.init_app_counter("app.pegasus",
-                                       name.c_str(),
-                                       COUNTER_TYPE_NUMBER_PERCENTILES,
-                                       "statistic the latency of INCR request");
-
-    name = fmt::format("check_and_set_latency@{}", str_gpid);
-    _pfc_check_and_set_latency.init_app_counter("app.pegasus",
-                                                name.c_str(),
-                                                COUNTER_TYPE_NUMBER_PERCENTILES,
-                                                "statistic the latency of CHECK_AND_SET request");
-
-    name = fmt::format("check_and_mutate_latency@{}", str_gpid);
-    _pfc_check_and_mutate_latency.init_app_counter(
-        "app.pegasus",
-        name.c_str(),
-        COUNTER_TYPE_NUMBER_PERCENTILES,
-        "statistic the latency of CHECK_AND_MUTATE request");
-
-    _pfc_duplicate_qps.init_app_counter("app.pegasus",
-                                        fmt::format("duplicate_qps@{}", str_gpid).c_str(),
-                                        COUNTER_TYPE_RATE,
-                                        "statistic the qps of DUPLICATE requests");
-
-    _pfc_dup_time_lag.init_app_counter(
-        "app.pegasus",
-        fmt::format("dup.time_lag_ms@{}", app_name()).c_str(),
-        COUNTER_TYPE_NUMBER_PERCENTILES,
-        "the time (in ms) lag between master and slave in the duplication");
-
-    _pfc_dup_lagging_writes.init_app_counter(
-        "app.pegasus",
-        fmt::format("dup.lagging_writes@{}", app_name()).c_str(),
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "the number of lagging writes (time lag larger than `dup_lagging_write_threshold_ms`)");
+    _dup_lagging_write_threshold_ms = dsn_config_get_value_int64(
+        "pegasus.server",
+        "dup_lagging_write_threshold_ms",
+        10 * 1000,
+        "If the duration that a write flows from master to slave is larger than this threshold, "
+        "the write is defined a lagging write.");
 }
 
 pegasus_write_service::~pegasus_write_service() {}
@@ -177,15 +192,15 @@ int pegasus_write_service::multi_put(const db_write_context &ctx,
                                      const dsn::apps::multi_put_request &update,
                                      dsn::apps::update_response &resp)
 {
-    uint64_t start_time = dsn_now_ns();
-    _pfc_multi_put_qps->increment();
+    METRIC_VAR_AUTO_LATENCY(multi_put_latency_ns);
+    METRIC_VAR_INCREMENT(multi_put_requests);
+
     int err = _impl->multi_put(ctx, update, resp);
 
     if (_server->is_primary()) {
         _cu_calculator->add_multi_put_cu(resp.error, update.hash_key, update.kvs);
     }
 
-    _pfc_multi_put_latency->set(dsn_now_ns() - start_time);
     return err;
 }
 
@@ -193,15 +208,15 @@ int pegasus_write_service::multi_remove(int64_t decree,
                                         const dsn::apps::multi_remove_request &update,
                                         dsn::apps::multi_remove_response &resp)
 {
-    uint64_t start_time = dsn_now_ns();
-    _pfc_multi_remove_qps->increment();
+    METRIC_VAR_AUTO_LATENCY(multi_remove_latency_ns);
+    METRIC_VAR_INCREMENT(multi_remove_requests);
+
     int err = _impl->multi_remove(decree, update, resp);
 
     if (_server->is_primary()) {
         _cu_calculator->add_multi_remove_cu(resp.error, update.hash_key, update.sort_keys);
     }
 
-    _pfc_multi_remove_latency->set(dsn_now_ns() - start_time);
     return err;
 }
 
@@ -209,15 +224,15 @@ int pegasus_write_service::incr(int64_t decree,
                                 const dsn::apps::incr_request &update,
                                 dsn::apps::incr_response &resp)
 {
-    uint64_t start_time = dsn_now_ns();
-    _pfc_incr_qps->increment();
+    METRIC_VAR_AUTO_LATENCY(incr_latency_ns);
+    METRIC_VAR_INCREMENT(incr_requests);
+
     int err = _impl->incr(decree, update, resp);
 
     if (_server->is_primary()) {
         _cu_calculator->add_incr_cu(resp.error, update.key);
     }
 
-    _pfc_incr_latency->set(dsn_now_ns() - start_time);
     return err;
 }
 
@@ -225,8 +240,9 @@ int pegasus_write_service::check_and_set(int64_t decree,
                                          const dsn::apps::check_and_set_request &update,
                                          dsn::apps::check_and_set_response &resp)
 {
-    uint64_t start_time = dsn_now_ns();
-    _pfc_check_and_set_qps->increment();
+    METRIC_VAR_AUTO_LATENCY(check_and_set_latency_ns);
+    METRIC_VAR_INCREMENT(check_and_set_requests);
+
     int err = _impl->check_and_set(decree, update, resp);
 
     if (_server->is_primary()) {
@@ -237,7 +253,6 @@ int pegasus_write_service::check_and_set(int64_t decree,
                                              update.set_value);
     }
 
-    _pfc_check_and_set_latency->set(dsn_now_ns() - start_time);
     return err;
 }
 
@@ -245,8 +260,9 @@ int pegasus_write_service::check_and_mutate(int64_t decree,
                                             const dsn::apps::check_and_mutate_request &update,
                                             dsn::apps::check_and_mutate_response &resp)
 {
-    uint64_t start_time = dsn_now_ns();
-    _pfc_check_and_mutate_qps->increment();
+    METRIC_VAR_AUTO_LATENCY(check_and_mutate_latency_ns);
+    METRIC_VAR_INCREMENT(check_and_mutate_requests);
+
     int err = _impl->check_and_mutate(decree, update, resp);
 
     if (_server->is_primary()) {
@@ -254,7 +270,6 @@ int pegasus_write_service::check_and_mutate(int64_t decree,
             resp.error, update.hash_key, update.check_sort_key, update.mutate_list);
     }
 
-    _pfc_check_and_mutate_latency->set(dsn_now_ns() - start_time);
     return err;
 }
 
@@ -272,8 +287,7 @@ int pegasus_write_service::batch_put(const db_write_context &ctx,
 {
     CHECK_GT_MSG(_batch_start_time, 0, "batch_put must be called after batch_prepare");
 
-    _batch_qps_perfcounters.push_back(_pfc_put_qps.get());
-    _batch_latency_perfcounters.push_back(_pfc_put_latency.get());
+    ++_put_batch_size;
     int err = _impl->batch_put(ctx, update, resp);
 
     if (_server->is_primary()) {
@@ -289,8 +303,7 @@ int pegasus_write_service::batch_remove(int64_t decree,
 {
     CHECK_GT_MSG(_batch_start_time, 0, "batch_remove must be called after batch_prepare");
 
-    _batch_qps_perfcounters.push_back(_pfc_remove_qps.get());
-    _batch_latency_perfcounters.push_back(_pfc_remove_latency.get());
+    ++_remove_batch_size;
     int err = _impl->batch_remove(decree, key, resp);
 
     if (_server->is_primary()) {
@@ -322,15 +335,21 @@ void pegasus_write_service::set_default_ttl(uint32_t ttl) { _impl->set_default_t
 
 void pegasus_write_service::clear_up_batch_states()
 {
-    uint64_t latency = dsn_now_ns() - _batch_start_time;
-    for (dsn::perf_counter *pfc : _batch_qps_perfcounters)
-        pfc->increment();
-    for (dsn::perf_counter *pfc : _batch_latency_perfcounters)
-        pfc->set(latency);
-
-    _batch_qps_perfcounters.clear();
-    _batch_latency_perfcounters.clear();
+#define PROCESS_WRITE_BATCH(op)                                                                    \
+    do {                                                                                           \
+        METRIC_VAR_INCREMENT_BY(op##_requests, static_cast<int64_t>(_##op##_batch_size));          \
+        METRIC_VAR_SET(op##_latency_ns, static_cast<size_t>(_##op##_batch_size), latency_ns);      \
+        _##op##_batch_size = 0;                                                                    \
+    } while (0)
+
+    auto latency_ns = static_cast<int64_t>(dsn_now_ns() - _batch_start_time);
+
+    PROCESS_WRITE_BATCH(put);
+    PROCESS_WRITE_BATCH(remove);
+
     _batch_start_time = 0;
+
+#undef PROCESS_WRITE_BATCH
 }
 
 int pegasus_write_service::duplicate(int64_t decree,
@@ -350,14 +369,13 @@ int pegasus_write_service::duplicate(int64_t decree,
             return empty_put(decree);
         }
 
-        _pfc_duplicate_qps->increment();
-        auto cleanup = dsn::defer([this, &request]() {
-            uint64_t latency_ms = (dsn_now_us() - request.timestamp) / 1000;
-            if (latency_ms > FLAGS_dup_lagging_write_threshold_ms) {
-                _pfc_dup_lagging_writes->increment();
-            }
-            _pfc_dup_time_lag->set(latency_ms);
-        });
+        METRIC_VAR_INCREMENT(dup_requests);
+        METRIC_VAR_AUTO_LATENCY(
+            dup_time_lag_ms, request.timestamp * 1000, [this](uint64_t latency) {
+                if (latency > _dup_lagging_write_threshold_ms) {
+                    METRIC_VAR_INCREMENT(dup_lagging_writes);
+                }
+            });
         dsn::message_ex *write =
             dsn::from_blob_to_received_msg(request.task_code, request.raw_message);
         bool is_delete = request.task_code == dsn::apps::RPC_RRDB_RRDB_MULTI_REMOVE ||
diff --git a/src/server/pegasus_write_service.h b/src/server/pegasus_write_service.h
index 9fb854ffd..9e79f9122 100644
--- a/src/server/pegasus_write_service.h
+++ b/src/server/pegasus_write_service.h
@@ -25,13 +25,11 @@
 
 #include "common//duplication_common.h"
 #include "common/common.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica_base.h"
 #include "utils/errors.h"
 
 namespace dsn {
 class blob;
-class perf_counter;
 namespace apps {
 class check_and_mutate_request;
 class check_and_mutate_response;
@@ -216,28 +214,29 @@ private:
 
     capacity_unit_calculator *_cu_calculator;
 
-    ::dsn::perf_counter_wrapper _pfc_put_qps;
-    ::dsn::perf_counter_wrapper _pfc_multi_put_qps;
-    ::dsn::perf_counter_wrapper _pfc_remove_qps;
-    ::dsn::perf_counter_wrapper _pfc_multi_remove_qps;
-    ::dsn::perf_counter_wrapper _pfc_incr_qps;
-    ::dsn::perf_counter_wrapper _pfc_check_and_set_qps;
-    ::dsn::perf_counter_wrapper _pfc_check_and_mutate_qps;
-    ::dsn::perf_counter_wrapper _pfc_duplicate_qps;
-    ::dsn::perf_counter_wrapper _pfc_dup_time_lag;
-    ::dsn::perf_counter_wrapper _pfc_dup_lagging_writes;
-
-    ::dsn::perf_counter_wrapper _pfc_put_latency;
-    ::dsn::perf_counter_wrapper _pfc_multi_put_latency;
-    ::dsn::perf_counter_wrapper _pfc_remove_latency;
-    ::dsn::perf_counter_wrapper _pfc_multi_remove_latency;
-    ::dsn::perf_counter_wrapper _pfc_incr_latency;
-    ::dsn::perf_counter_wrapper _pfc_check_and_set_latency;
-    ::dsn::perf_counter_wrapper _pfc_check_and_mutate_latency;
-
-    // Records all requests.
-    std::vector<::dsn::perf_counter *> _batch_qps_perfcounters;
-    std::vector<::dsn::perf_counter *> _batch_latency_perfcounters;
+    METRIC_VAR_DECLARE_counter(put_requests);
+    METRIC_VAR_DECLARE_counter(multi_put_requests);
+    METRIC_VAR_DECLARE_counter(remove_requests);
+    METRIC_VAR_DECLARE_counter(multi_remove_requests);
+    METRIC_VAR_DECLARE_counter(incr_requests);
+    METRIC_VAR_DECLARE_counter(check_and_set_requests);
+    METRIC_VAR_DECLARE_counter(check_and_mutate_requests);
+
+    METRIC_VAR_DECLARE_percentile_int64(put_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(multi_put_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(remove_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(multi_remove_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(incr_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(check_and_set_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(check_and_mutate_latency_ns);
+
+    METRIC_VAR_DECLARE_counter(dup_requests);
+    METRIC_VAR_DECLARE_percentile_int64(dup_time_lag_ms);
+    METRIC_VAR_DECLARE_counter(dup_lagging_writes);
+
+    // Record batch size for put and remove requests.
+    uint32_t _put_batch_size;
+    uint32_t _remove_batch_size;
 
     // TODO(wutao1): add perf counters for failed rpc.
 };
diff --git a/src/server/test/pegasus_server_write_test.cpp b/src/server/test/pegasus_server_write_test.cpp
index 0e8c57114..b771e67bf 100644
--- a/src/server/test/pegasus_server_write_test.cpp
+++ b/src/server/test/pegasus_server_write_test.cpp
@@ -108,8 +108,8 @@ public:
                 // make sure everything is cleanup after batch write.
                 ASSERT_TRUE(_server_write->_put_rpc_batch.empty());
                 ASSERT_TRUE(_server_write->_remove_rpc_batch.empty());
-                ASSERT_TRUE(_server_write->_write_svc->_batch_qps_perfcounters.empty());
-                ASSERT_TRUE(_server_write->_write_svc->_batch_latency_perfcounters.empty());
+                ASSERT_EQ(_server_write->_write_svc->_put_batch_size, 0);
+                ASSERT_EQ(_server_write->_write_svc->_remove_batch_size, 0);
                 ASSERT_EQ(_server_write->_write_svc->_batch_start_time, 0);
                 ASSERT_EQ(_server_write->_write_svc->_impl->_rocksdb_wrapper->_write_batch->Count(),
                           0);
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index 27c6355f3..2d6da6c0f 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -51,6 +51,7 @@
 #include "utils/singleton.h"
 #include "utils/string_view.h"
 #include "utils/synchronize.h"
+#include "utils/time_utils.h"
 
 namespace boost {
 namespace system {
@@ -89,7 +90,8 @@ class error_code;
 // Instantiating the metric in whatever class represents it with some initial arguments, if any:
 // metric_instance = METRIC_my_gauge_name.instantiate(entity_instance, ...);
 
-// Convenient macros are provided to define entity types and metric prototypes.
+// The following are convenient macros provided to define entity types and metric prototypes.
+
 #define METRIC_DEFINE_entity(name) ::dsn::metric_entity_prototype METRIC_ENTITY_##name(#name)
 #define METRIC_DEFINE_gauge_int64(entity_type, name, unit, desc, ...)                              \
     ::dsn::gauge_prototype<int64_t> METRIC_##name(                                                 \
@@ -97,6 +99,7 @@ class error_code;
 #define METRIC_DEFINE_gauge_double(entity_type, name, unit, desc, ...)                             \
     ::dsn::gauge_prototype<double> METRIC_##name(                                                  \
         {#entity_type, dsn::metric_type::kGauge, #name, unit, desc, ##__VA_ARGS__})
+
 // There are 2 kinds of counters:
 // - `counter` is the general type of counter that is implemented by striped_long_adder, which can
 //   achieve high performance while consuming less memory if it's not updated very frequently.
@@ -141,6 +144,42 @@ class error_code;
 #define METRIC_DECLARE_percentile_double(name)                                                     \
     extern dsn::floating_percentile_prototype<double> METRIC_##name
 
+// Following METRIC_*VAR* macros are introduced so that:
+// * only need to use prototype name to operate each metric variable;
+// * uniformly name each variable in user class;
+// * differentiate operations on metrics significantly from main logic, improving code readability.
+
+// Declare a metric variable in user class.
+//
+// Since a type tends to be a class template where there might be commas, use variadic arguments
+// instead of a single fixed argument to represent a type.
+#define METRIC_VAR_DECLARE(name, ...) __VA_ARGS__ _##name
+#define METRIC_VAR_DECLARE_gauge_int64(name) METRIC_VAR_DECLARE(name, dsn::gauge_ptr<int64_t>)
+#define METRIC_VAR_DECLARE_counter(name)                                                           \
+    METRIC_VAR_DECLARE(name, dsn::counter_ptr<dsn::striped_long_adder, false>)
+#define METRIC_VAR_DECLARE_percentile_int64(name)                                                  \
+    METRIC_VAR_DECLARE(name, dsn::percentile_ptr<int64_t>)
+
+// Initialize a metric variable in user class.
+#define METRIC_VAR_INIT(name, entity) _##name(METRIC_##name.instantiate(entity##_metric_entity()))
+#define METRIC_VAR_INIT_replica(name) METRIC_VAR_INIT(name, replica)
+
+// Perform increment-related operations on metrics including gauge and counter.
+#define METRIC_VAR_INCREMENT_BY(name, x) _##name->increment_by(x)
+#define METRIC_VAR_INCREMENT(name) _##name->increment()
+
+// Perform set() operations on metrics including gauge and percentile.
+//
+// There are 2 kinds of invocations of set() for a metric:
+// * set(val): set a single value for a metric, such as gauge, percentile;
+// * set(n, val): set multiple repeated values (the number of duplicates is n) for a metric,
+// such as percentile.
+#define METRIC_VAR_SET(name, ...) _##name->set(__VA_ARGS__)
+
+// Convenient macro that is used to compute latency automatically, which is dedicated to percentile.
+#define METRIC_VAR_AUTO_LATENCY(name, ...)                                                         \
+    dsn::auto_latency __##name##_auto_latency(_##name, ##__VA_ARGS__)
+
 namespace dsn {
 class metric;                  // IWYU pragma: keep
 class metric_entity_prototype; // IWYU pragma: keep
@@ -552,7 +591,7 @@ ENUM_REG_WITH_CUSTOM_NAME(metric_type::kVolatileCounter, volatile_counter)
 ENUM_REG_WITH_CUSTOM_NAME(metric_type::kPercentile, percentile)
 ENUM_END(metric_type)
 
-enum class metric_unit
+enum class metric_unit : size_t
 {
     kNanoSeconds,
     kMicroSeconds,
@@ -562,6 +601,31 @@ enum class metric_unit
     kInvalidUnit,
 };
 
+#define METRIC_ASSERT_UNIT_LATENCY(unit, index)                                                    \
+    static_assert(static_cast<size_t>(metric_unit::unit) == index,                                 \
+                  #unit " should be at index " #index)
+
+METRIC_ASSERT_UNIT_LATENCY(kNanoSeconds, 0);
+METRIC_ASSERT_UNIT_LATENCY(kMicroSeconds, 1);
+METRIC_ASSERT_UNIT_LATENCY(kMilliSeconds, 2);
+METRIC_ASSERT_UNIT_LATENCY(kSeconds, 3);
+
+const std::vector<uint64_t> kMetricLatencyConverterFromNS = {
+    1, 1000, 1000 * 1000, 1000 * 1000 * 1000};
+
+inline uint64_t convert_metric_latency_from_ns(uint64_t latency_ns, metric_unit target_unit)
+{
+    if (dsn_likely(target_unit == metric_unit::kNanoSeconds)) {
+        // Since nanoseconds are used as the latency unit more frequently, eliminate unnecessary
+        // conversion by branch prediction.
+        return latency_ns;
+    }
+
+    auto index = static_cast<size_t>(target_unit);
+    CHECK_LT(index, kMetricLatencyConverterFromNS.size());
+    return latency_ns / kMetricLatencyConverterFromNS[index];
+}
+
 ENUM_BEGIN(metric_unit, metric_unit::kInvalidUnit)
 ENUM_REG_WITH_CUSTOM_NAME(metric_unit::kNanoSeconds, nanoseconds)
 ENUM_REG_WITH_CUSTOM_NAME(metric_unit::kMicroSeconds, microseconds)
@@ -1066,6 +1130,13 @@ public:
         _samples.get()[index & (_sample_size - 1)] = val;
     }
 
+    void set(size_t n, const value_type &val)
+    {
+        for (size_t i = 0; i < n; ++i) {
+            set(val);
+        }
+    }
+
     // If `type` is not configured, it will return false with zero value stored in `val`;
     // otherwise, it will always return true with the value corresponding to `type`.
     bool get(kth_percentile_type type, value_type &val) const
@@ -1177,6 +1248,7 @@ private:
 
     friend class metric_entity;
     friend class ref_ptr<percentile<value_type, NthElementFinder>>;
+    friend class MetricVarTest;
 
     virtual void close() override
     {
@@ -1199,6 +1271,20 @@ private:
         release_ref();
     }
 
+    std::vector<value_type> samples_for_test()
+    {
+        size_type real_sample_size = std::min(static_cast<size_type>(_tail.load()), _sample_size);
+        if (real_sample_size == 0) {
+            return std::vector<value_type>();
+        }
+
+        std::vector<value_type> real_samples(real_sample_size);
+        std::copy(_samples.get(), _samples.get() + real_sample_size, real_samples.begin());
+        return real_samples;
+    }
+
+    void reset_tail_for_test() { _tail.store(0); }
+
     value_type value(size_t index) const
     {
         return _full_nth_elements[index].load(std::memory_order_relaxed);
@@ -1219,7 +1305,7 @@ private:
         }
 
         // Find nth elements.
-        std::vector<T> array(real_sample_size);
+        std::vector<value_type> array(real_sample_size);
         std::copy(_samples.get(), _samples.get() + real_sample_size, array.begin());
         _nth_element_finder(array.begin(), array.begin(), array.end());
 
@@ -1288,4 +1374,47 @@ template <typename T,
 using floating_percentile_prototype =
     metric_prototype_with<floating_percentile<T, NthElementFinder>>;
 
+// Compute latency automatically at the end of the scope, which is set to percentile which it has
+// bound to.
+class auto_latency
+{
+public:
+    auto_latency(const percentile_ptr<int64_t> &percentile) : _percentile(percentile) {}
+
+    auto_latency(const percentile_ptr<int64_t> &percentile, std::function<void(uint64_t)> callback)
+        : _percentile(percentile), _callback(std::move(callback))
+    {
+    }
+
+    auto_latency(const percentile_ptr<int64_t> &percentile, uint64_t start_time_ns)
+        : _percentile(percentile), _chrono(start_time_ns)
+    {
+    }
+
+    auto_latency(const percentile_ptr<int64_t> &percentile,
+                 uint64_t start_time_ns,
+                 std::function<void(uint64_t)> callback)
+        : _percentile(percentile), _chrono(start_time_ns), _callback(std::move(callback))
+    {
+    }
+
+    ~auto_latency()
+    {
+        auto latency =
+            convert_metric_latency_from_ns(_chrono.duration_ns(), _percentile->prototype()->unit());
+        _percentile->set(static_cast<int64_t>(latency));
+
+        if (_callback) {
+            _callback(latency);
+        }
+    }
+
+private:
+    percentile_ptr<int64_t> _percentile;
+    utils::chronograph _chrono;
+    std::function<void(uint64_t)> _callback;
+
+    DISALLOW_COPY_AND_ASSIGN(auto_latency);
+};
+
 } // namespace dsn
diff --git a/src/utils/test/metrics_test.cpp b/src/utils/test/metrics_test.cpp
index b81a0ac8e..670af36b7 100644
--- a/src/utils/test/metrics_test.cpp
+++ b/src/utils/test/metrics_test.cpp
@@ -83,6 +83,8 @@ METRIC_DEFINE_entity(my_server);
 METRIC_DEFINE_entity(my_table);
 METRIC_DEFINE_entity(my_replica);
 
+#define METRIC_VAR_INIT_my_replica(name) METRIC_VAR_INIT(name, my_replica)
+
 // Dedicated entity for getting metrics by http service.
 METRIC_DEFINE_entity(my_app);
 
@@ -174,6 +176,26 @@ METRIC_DEFINE_percentile_double(my_server,
                                 dsn::metric_unit::kNanoSeconds,
                                 "a server-level percentile of double type for test");
 
+METRIC_DEFINE_percentile_int64(my_replica,
+                               test_replica_percentile_int64_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "a replica-level percentile of int64 type in nanoseconds for test");
+
+METRIC_DEFINE_percentile_int64(my_replica,
+                               test_replica_percentile_int64_us,
+                               dsn::metric_unit::kMicroSeconds,
+                               "a replica-level percentile of int64 type in microseconds for test");
+
+METRIC_DEFINE_percentile_int64(my_replica,
+                               test_replica_percentile_int64_ms,
+                               dsn::metric_unit::kMilliSeconds,
+                               "a replica-level percentile of int64 type in milliseconds for test");
+
+METRIC_DEFINE_percentile_int64(my_replica,
+                               test_replica_percentile_int64_s,
+                               dsn::metric_unit::kSeconds,
+                               "a replica-level percentile of int64 type in seconds for test");
+
 namespace dsn {
 
 TEST(metrics_test, create_entity)
@@ -737,9 +759,7 @@ void run_percentile(const metric_entity_ptr &my_entity,
     auto my_metric = prototype.instantiate(my_entity, interval_ms, kth_percentiles, sample_size);
 
     // Preload zero in current thread.
-    for (size_t i = 0; i < num_preload; ++i) {
-        my_metric->set(0);
-    }
+    my_metric->set(num_preload, 0);
 
     // Load other data in each spawned thread evenly.
     const size_t num_operations = data.size() / num_threads;
@@ -3056,4 +3076,125 @@ INSTANTIATE_TEST_CASE_P(MetricsTest,
                         MetricsRetirementTest,
                         testing::ValuesIn(metrics_retirement_tests));
 
+class MetricVarTest : public testing::Test
+{
+protected:
+    MetricVarTest();
+
+    void SetUp() override
+    {
+        _test_replica_gauge_int64->set(0);
+        _test_replica_counter->reset();
+        _test_replica_percentile_int64_ns->reset_tail_for_test();
+        _test_replica_percentile_int64_us->reset_tail_for_test();
+        _test_replica_percentile_int64_ms->reset_tail_for_test();
+        _test_replica_percentile_int64_s->reset_tail_for_test();
+    }
+
+    const metric_entity_ptr &my_replica_metric_entity() const { return _my_replica_metric_entity; }
+
+    void test_set_percentile(const std::vector<int64_t> &expected_samples);
+    void test_set_percentile(size_t n, int64_t val);
+
+    const metric_entity_ptr _my_replica_metric_entity;
+    METRIC_VAR_DECLARE_gauge_int64(test_replica_gauge_int64);
+    METRIC_VAR_DECLARE_counter(test_replica_counter);
+    METRIC_VAR_DECLARE_percentile_int64(test_replica_percentile_int64_ns);
+    METRIC_VAR_DECLARE_percentile_int64(test_replica_percentile_int64_us);
+    METRIC_VAR_DECLARE_percentile_int64(test_replica_percentile_int64_ms);
+    METRIC_VAR_DECLARE_percentile_int64(test_replica_percentile_int64_s);
+
+    DISALLOW_COPY_AND_ASSIGN(MetricVarTest);
+};
+
+MetricVarTest::MetricVarTest()
+    : _my_replica_metric_entity(METRIC_ENTITY_my_replica.instantiate("replica_var_test")),
+      METRIC_VAR_INIT_my_replica(test_replica_gauge_int64),
+      METRIC_VAR_INIT_my_replica(test_replica_counter),
+      METRIC_VAR_INIT_my_replica(test_replica_percentile_int64_ns),
+      METRIC_VAR_INIT_my_replica(test_replica_percentile_int64_us),
+      METRIC_VAR_INIT_my_replica(test_replica_percentile_int64_ms),
+      METRIC_VAR_INIT_my_replica(test_replica_percentile_int64_s)
+{
+}
+
+#define METRIC_VAR_SAMPLES(name) _##name->samples_for_test()
+
+void MetricVarTest::test_set_percentile(const std::vector<int64_t> &expected_samples)
+{
+    for (const auto &val : expected_samples) {
+        METRIC_VAR_SET(test_replica_percentile_int64_ns, val);
+    }
+    EXPECT_EQ(expected_samples, METRIC_VAR_SAMPLES(test_replica_percentile_int64_ns));
+}
+
+void MetricVarTest::test_set_percentile(size_t n, int64_t val)
+{
+    METRIC_VAR_SET(test_replica_percentile_int64_ns, n, val);
+    EXPECT_EQ(std::vector<int64_t>(n, val), METRIC_VAR_SAMPLES(test_replica_percentile_int64_ns));
+}
+
+#define METRIC_VAR_VALUE(name) _##name->value()
+
+#define TEST_METRIC_VAR_INCREMENT(name)                                                            \
+    do {                                                                                           \
+        ASSERT_EQ(0, METRIC_VAR_VALUE(name));                                                      \
+                                                                                                   \
+        METRIC_VAR_INCREMENT(name);                                                                \
+        ASSERT_EQ(1, METRIC_VAR_VALUE(name));                                                      \
+                                                                                                   \
+        METRIC_VAR_INCREMENT(name);                                                                \
+        ASSERT_EQ(2, METRIC_VAR_VALUE(name));                                                      \
+                                                                                                   \
+        METRIC_VAR_INCREMENT_BY(name, 5);                                                          \
+        ASSERT_EQ(7, METRIC_VAR_VALUE(name));                                                      \
+                                                                                                   \
+        METRIC_VAR_INCREMENT_BY(name, 18);                                                         \
+        ASSERT_EQ(25, METRIC_VAR_VALUE(name));                                                     \
+    } while (0);
+
+TEST_F(MetricVarTest, IncrementGauge) { TEST_METRIC_VAR_INCREMENT(test_replica_gauge_int64); }
+
+TEST_F(MetricVarTest, IncrementCounter) { TEST_METRIC_VAR_INCREMENT(test_replica_counter); }
+
+TEST_F(MetricVarTest, SetGauge)
+{
+    ASSERT_EQ(0, METRIC_VAR_VALUE(test_replica_gauge_int64));
+
+    METRIC_VAR_SET(test_replica_gauge_int64, 5);
+    ASSERT_EQ(5, METRIC_VAR_VALUE(test_replica_gauge_int64));
+
+    METRIC_VAR_SET(test_replica_gauge_int64, 18);
+    ASSERT_EQ(18, METRIC_VAR_VALUE(test_replica_gauge_int64));
+}
+
+TEST_F(MetricVarTest, SetPercentileIndividually) { test_set_percentile({20, 50, 10, 25, 16}); }
+
+TEST_F(MetricVarTest, SetPercentileRepeatedly) { test_set_percentile(5, 100); }
+
+#define TEST_METRIC_VAR_AUTO_LATENCY(unit_abbr, factor)                                            \
+    do {                                                                                           \
+        auto start_time_ns = dsn_now_ns();                                                         \
+        uint64_t actual_latency_ns = 0;                                                            \
+        {                                                                                          \
+            METRIC_VAR_AUTO_LATENCY(test_replica_percentile_int64_##unit_abbr,                     \
+                                    start_time_ns,                                                 \
+                                    [&actual_latency_ns](uint64_t latency) mutable {               \
+                                        actual_latency_ns = latency * factor;                      \
+                                    });                                                            \
+        }                                                                                          \
+                                                                                                   \
+        uint64_t expected_latency_ns = dsn_now_ns() - start_time_ns;                               \
+        ASSERT_GE(expected_latency_ns, actual_latency_ns);                                         \
+        EXPECT_LT(expected_latency_ns - actual_latency_ns, 1000 * 1000);                           \
+    } while (0)
+
+TEST_F(MetricVarTest, AutoLatencyNanoSeconds) { TEST_METRIC_VAR_AUTO_LATENCY(ns, 1); }
+
+TEST_F(MetricVarTest, AutoLatencyMicroSeconds) { TEST_METRIC_VAR_AUTO_LATENCY(us, 1000); }
+
+TEST_F(MetricVarTest, AutoLatencyMilliSeconds) { TEST_METRIC_VAR_AUTO_LATENCY(ms, 1000 * 1000); }
+
+TEST_F(MetricVarTest, AutoLatencySeconds) { TEST_METRIC_VAR_AUTO_LATENCY(s, 1000 * 1000 * 1000); }
+
 } // namespace dsn
diff --git a/src/utils/time_utils.h b/src/utils/time_utils.h
index 50df0b785..1db1676b0 100644
--- a/src/utils/time_utils.h
+++ b/src/utils/time_utils.h
@@ -32,7 +32,10 @@
 #include <cstdio>
 #include <string>
 
-#include "string_view.h"
+#include "runtime/api_layer1.h"
+#include "utils/fmt_logging.h"
+#include "utils/ports.h"
+#include "utils/string_view.h"
 
 namespace dsn {
 namespace utils {
@@ -130,5 +133,28 @@ inline int64_t hh_mm_today_to_unix_sec(string_view hhmm_of_day)
     return get_unix_sec_today_midnight() + sec_of_day;
 }
 
+class chronograph
+{
+public:
+    chronograph() : chronograph(dsn_now_ns()) {}
+    chronograph(uint64_t start_time_ns) : _start_time_ns(start_time_ns) {}
+    ~chronograph() = default;
+
+    inline void reset_start_time() { _start_time_ns = dsn_now_ns(); }
+
+    inline uint64_t duration_ns()
+    {
+        auto now = dsn_now_ns();
+        CHECK_GE(now, _start_time_ns);
+
+        return now - _start_time_ns;
+    }
+
+private:
+    uint64_t _start_time_ns;
+
+    DISALLOW_COPY_AND_ASSIGN(chronograph);
+};
+
 } // namespace utils
 } // namespace dsn


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


[incubator-pegasus] 13/23: feat(new_metrics): add table-level metric entity and migrate table-level metrics for server_state of meta (#1431)

Posted by wa...@apache.org.
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 db1d82f58a7145121f1ed0256b932b31763801a9
Author: Dan Wang <wa...@apache.org>
AuthorDate: Tue Apr 11 00:08:03 2023 +0800

    feat(new_metrics): add table-level metric entity and migrate table-level metrics for server_state of meta (#1431)
    
    https://github.com/apache/incubator-pegasus/issues/1331
    
    In perf counters, all metrics of server_state are server-level, for example,
    the number of healthy partitions among all tables of a pegasus cluster.
    
    However, sometimes this is not enough. For example, the metric shows
    that there are 4 unwritable partitions: the 4 unwritable partitions might
    belong to different tables; or, they might belong to one table.
    
    Therefore, these server-level metrics could be changed to table-level.
    This will provide us with the status of each table. On the other hand,
    once server-level metrics is needed, just aggregate on table-level ones.
    
    The metrics of server_state that are migrated and changed to table-level
    include: The number of dead, unreadable, unwritable, writable-ill, and
    healthy partitions among all partitions of a table, the number of times
    the configuration has been changed and the number of times the status
    of partition has been changed to unwritable or writable for a table.
    
    To implement table-level metrics, table-level metric entity is also added.
---
 src/meta/server_state.cpp                   |  87 ++++++-------
 src/meta/server_state.h                     |  13 +-
 src/meta/server_state_restore.cpp           |   2 +
 src/meta/table_metrics.cpp                  | 187 ++++++++++++++++++++++++++++
 src/meta/table_metrics.h                    | 136 ++++++++++++++++++++
 src/meta/test/state_sync_test.cpp           |   2 +
 src/meta/test/update_configuration_test.cpp |   2 +
 src/utils/metrics.h                         |  10 +-
 8 files changed, 376 insertions(+), 63 deletions(-)

diff --git a/src/meta/server_state.cpp b/src/meta/server_state.cpp
index ecf66a92b..71fe7b1af 100644
--- a/src/meta/server_state.cpp
+++ b/src/meta/server_state.cpp
@@ -62,10 +62,10 @@
 #include "meta/meta_service.h"
 #include "meta/meta_state_service.h"
 #include "meta/partition_guardian.h"
+#include "meta/table_metrics.h"
 #include "meta_admin_types.h"
 #include "meta_bulk_load_service.h"
 #include "metadata_types.h"
-#include "perf_counter/perf_counter.h"
 #include "replica_admin_types.h"
 #include "runtime/api_layer1.h"
 #include "runtime/rpc/rpc_address.h"
@@ -85,12 +85,9 @@
 #include "utils/config_api.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
-#include "utils/singleton.h"
 #include "utils/string_conv.h"
 #include "utils/strings.h"
 
-using namespace dsn;
-
 namespace dsn {
 namespace replication {
 DSN_DEFINE_bool(meta_server,
@@ -214,41 +211,6 @@ void server_state::initialize(meta_service *meta_svc, const std::string &apps_ro
     _apps_root = apps_root;
     _add_secondary_enable_flow_control = FLAGS_add_secondary_enable_flow_control;
     _add_secondary_max_count_for_one_node = FLAGS_add_secondary_max_count_for_one_node;
-
-    _dead_partition_count.init_app_counter("eon.server_state",
-                                           "dead_partition_count",
-                                           COUNTER_TYPE_NUMBER,
-                                           "current dead partition count");
-    _unreadable_partition_count.init_app_counter("eon.server_state",
-                                                 "unreadable_partition_count",
-                                                 COUNTER_TYPE_NUMBER,
-                                                 "current unreadable partition count");
-    _unwritable_partition_count.init_app_counter("eon.server_state",
-                                                 "unwritable_partition_count",
-                                                 COUNTER_TYPE_NUMBER,
-                                                 "current unwritable partition count");
-    _writable_ill_partition_count.init_app_counter("eon.server_state",
-                                                   "writable_ill_partition_count",
-                                                   COUNTER_TYPE_NUMBER,
-                                                   "current writable ill partition count");
-    _healthy_partition_count.init_app_counter("eon.server_state",
-                                              "healthy_partition_count",
-                                              COUNTER_TYPE_NUMBER,
-                                              "current healthy partition count");
-    _recent_update_config_count.init_app_counter("eon.server_state",
-                                                 "recent_update_config_count",
-                                                 COUNTER_TYPE_VOLATILE_NUMBER,
-                                                 "update configuration count in the recent period");
-    _recent_partition_change_unwritable_count.init_app_counter(
-        "eon.server_state",
-        "recent_partition_change_unwritable_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "partition change to unwritable count in the recent period");
-    _recent_partition_change_writable_count.init_app_counter(
-        "eon.server_state",
-        "recent_partition_change_writable_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "partition change to writable count in the recent period");
 }
 
 bool server_state::spin_wait_staging(int timeout_seconds)
@@ -529,12 +491,14 @@ error_code server_state::initialize_default_apps()
 error_code server_state::sync_apps_to_remote_storage()
 {
     _exist_apps.clear();
+    _table_metric_entities.clear_entities();
     for (auto &kv_pair : _all_apps) {
         if (kv_pair.second->status == app_status::AS_CREATING) {
             CHECK(_exist_apps.find(kv_pair.second->app_name) == _exist_apps.end(),
                   "invalid app name, name = {}",
                   kv_pair.second->app_name);
             _exist_apps.emplace(kv_pair.second->app_name, kv_pair.second);
+            _table_metric_entities.create_entity(kv_pair.first);
         }
     }
 
@@ -584,6 +548,7 @@ error_code server_state::sync_apps_to_remote_storage()
 
     if (err != ERR_OK) {
         _exist_apps.clear();
+        _table_metric_entities.clear_entities();
         return err;
     }
     for (auto &kv : _all_apps) {
@@ -699,6 +664,7 @@ dsn::error_code server_state::sync_apps_from_remote_storage()
                         if (app->status == app_status::AS_AVAILABLE) {
                             app->status = app_status::AS_CREATING;
                             _exist_apps.emplace(app->app_name, app);
+                            _table_metric_entities.create_entity(app->app_id);
                         } else if (app->status == app_status::AS_DROPPED) {
                             app->status = app_status::AS_DROPPING;
                         } else {
@@ -726,6 +692,7 @@ dsn::error_code server_state::sync_apps_from_remote_storage()
 
     _all_apps.clear();
     _exist_apps.clear();
+    _table_metric_entities.clear_entities();
 
     std::string transaction_state;
     storage
@@ -1197,6 +1164,7 @@ void server_state::create_app(dsn::message_ex *msg)
 
             _all_apps.emplace(app->app_id, app);
             _exist_apps.emplace(request.app_name, app);
+            _table_metric_entities.create_entity(app->app_id);
         }
     }
 
@@ -1214,6 +1182,7 @@ void server_state::do_app_drop(std::shared_ptr<app_state> &app)
         if (ERR_OK == ec) {
             zauto_write_lock l(_lock);
             _exist_apps.erase(app->app_name);
+            _table_metric_entities.remove_entity(app->app_id);
             for (int i = 0; i < app->partition_count; ++i) {
                 drop_partition(app, i);
             }
@@ -1425,6 +1394,7 @@ void server_state::recall_app(dsn::message_ex *msg)
                     target_app->helpers->pending_response = msg;
 
                     _exist_apps.emplace(target_app->app_name, target_app);
+                    _table_metric_entities.create_entity(target_app->app_id);
                 }
             }
         }
@@ -1465,7 +1435,7 @@ void server_state::send_proposal(rpc_address target, const configuration_update_
              proposal.node);
     dsn::message_ex *msg =
         dsn::message_ex::create_request(RPC_CONFIG_PROPOSAL, 0, proposal.config.pid.thread_hash());
-    ::marshall(msg, proposal);
+    dsn::marshall(msg, proposal);
     _meta_svc->send_message(target, msg);
 }
 
@@ -1649,12 +1619,15 @@ void server_state::update_configuration_locally(
         _config_change_subscriber(_all_apps);
     }
 
-    _recent_update_config_count->increment();
+    METRIC_CALL_TABLE_INCREMENT_METHOD(
+        _table_metric_entities, partition_configuration_changes, app.app_id);
     if (old_health_status >= HS_WRITABLE_ILL && new_health_status < HS_WRITABLE_ILL) {
-        _recent_partition_change_unwritable_count->increment();
+        METRIC_CALL_TABLE_INCREMENT_METHOD(
+            _table_metric_entities, unwritable_partition_changes, app.app_id);
     }
     if (old_health_status < HS_WRITABLE_ILL && new_health_status >= HS_WRITABLE_ILL) {
-        _recent_partition_change_writable_count->increment();
+        METRIC_CALL_TABLE_INCREMENT_METHOD(
+            _table_metric_entities, writable_partition_changes, app.app_id);
     }
 }
 
@@ -2446,25 +2419,35 @@ bool server_state::can_run_balancer()
     return true;
 }
 
-void server_state::update_partition_perf_counter()
+void server_state::update_partition_metrics()
 {
-    int counters[HS_MAX_VALUE];
-    ::memset(counters, 0, sizeof(counters));
     auto func = [&](const std::shared_ptr<app_state> &app) {
+        int counters[HS_MAX_VALUE] = {0};
+
         int min_2pc_count =
             _meta_svc->get_options().app_mutation_2pc_min_replica_count(app->max_replica_count);
         for (unsigned int i = 0; i != app->partition_count; ++i) {
             health_status st = partition_health_status(app->partitions[i], min_2pc_count);
             counters[st]++;
         }
+
+        METRIC_CALL_TABLE_SET_METHOD(
+            _table_metric_entities, dead_partitions, app->app_id, counters[HS_DEAD]);
+        METRIC_CALL_TABLE_SET_METHOD(
+            _table_metric_entities, unreadable_partitions, app->app_id, counters[HS_UNREADABLE]);
+        METRIC_CALL_TABLE_SET_METHOD(
+            _table_metric_entities, unwritable_partitions, app->app_id, counters[HS_UNWRITABLE]);
+        METRIC_CALL_TABLE_SET_METHOD(_table_metric_entities,
+                                     writable_ill_partitions,
+                                     app->app_id,
+                                     counters[HS_WRITABLE_ILL]);
+        METRIC_CALL_TABLE_SET_METHOD(
+            _table_metric_entities, healthy_partitions, app->app_id, counters[HS_HEALTHY]);
+
         return true;
     };
+
     for_each_available_app(_all_apps, func);
-    _dead_partition_count->set(counters[HS_DEAD]);
-    _unreadable_partition_count->set(counters[HS_UNREADABLE]);
-    _unwritable_partition_count->set(counters[HS_UNWRITABLE]);
-    _writable_ill_partition_count->set(counters[HS_WRITABLE_ILL]);
-    _healthy_partition_count->set(counters[HS_HEALTHY]);
 }
 
 bool server_state::check_all_partitions()
@@ -2475,7 +2458,7 @@ bool server_state::check_all_partitions()
 
     zauto_write_lock l(_lock);
 
-    update_partition_perf_counter();
+    update_partition_metrics();
 
     // first the cure stage
     if (level <= meta_function_level::fl_freezed) {
diff --git a/src/meta/server_state.h b/src/meta/server_state.h
index a3feb28f9..13a8ef8bd 100644
--- a/src/meta/server_state.h
+++ b/src/meta/server_state.h
@@ -52,9 +52,9 @@
 #include "dsn.layer2_types.h"
 #include "meta/meta_rpc_types.h"
 #include "meta_data.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "runtime/task/task.h"
 #include "runtime/task/task_tracker.h"
+#include "table_metrics.h"
 #include "utils/error_code.h"
 #include "utils/zlocks.h"
 
@@ -230,7 +230,7 @@ private:
     bool can_run_balancer();
 
     // user should lock it first
-    void update_partition_perf_counter();
+    void update_partition_metrics();
 
     error_code dump_app_states(const char *local_path,
                                const std::function<app_state *()> &iterator);
@@ -437,14 +437,7 @@ private:
     int32_t _add_secondary_max_count_for_one_node;
     std::vector<std::unique_ptr<command_deregister>> _cmds;
 
-    perf_counter_wrapper _dead_partition_count;
-    perf_counter_wrapper _unreadable_partition_count;
-    perf_counter_wrapper _unwritable_partition_count;
-    perf_counter_wrapper _writable_ill_partition_count;
-    perf_counter_wrapper _healthy_partition_count;
-    perf_counter_wrapper _recent_update_config_count;
-    perf_counter_wrapper _recent_partition_change_unwritable_count;
-    perf_counter_wrapper _recent_partition_change_writable_count;
+    table_metric_entities _table_metric_entities;
 };
 
 } // namespace replication
diff --git a/src/meta/server_state_restore.cpp b/src/meta/server_state_restore.cpp
index 93d790e66..f513b0c4a 100644
--- a/src/meta/server_state_restore.cpp
+++ b/src/meta/server_state_restore.cpp
@@ -37,6 +37,7 @@
 #include "dsn.layer2_types.h"
 #include "meta/meta_data.h"
 #include "meta/meta_rpc_types.h"
+#include "meta/table_metrics.h"
 #include "meta_admin_types.h"
 #include "meta_service.h"
 #include "runtime/rpc/rpc_address.h"
@@ -147,6 +148,7 @@ std::pair<dsn::error_code, std::shared_ptr<app_state>> server_state::restore_app
 
             _all_apps.emplace(app->app_id, app);
             _exist_apps.emplace(info.app_name, app);
+            _table_metric_entities.create_entity(app->app_id);
         }
     }
     // TODO: using one single env to replace
diff --git a/src/meta/table_metrics.cpp b/src/meta/table_metrics.cpp
new file mode 100644
index 000000000..511be89eb
--- /dev/null
+++ b/src/meta/table_metrics.cpp
@@ -0,0 +1,187 @@
+// 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 "table_metrics.h"
+
+#include <fmt/core.h>
+#include <string>
+
+#include "utils/fmt_logging.h"
+#include "utils/string_view.h"
+
+METRIC_DEFINE_entity(table);
+
+// The number of partitions in each status, see `health_status` and `partition_health_status()`
+// for details.
+
+METRIC_DEFINE_gauge_int64(
+    table,
+    dead_partitions,
+    dsn::metric_unit::kPartitions,
+    "The number of dead partitions, which means primary = 0 && secondary = 0");
+
+METRIC_DEFINE_gauge_int64(table,
+                          unreadable_partitions,
+                          dsn::metric_unit::kPartitions,
+                          "The number of unreadable partitions, which means primary = 0 && "
+                          "secondary > 0");
+
+METRIC_DEFINE_gauge_int64(table,
+                          unwritable_partitions,
+                          dsn::metric_unit::kPartitions,
+                          "The number of unwritable partitions, which means primary = 1 && "
+                          "primary + secondary < mutation_2pc_min_replica_count");
+
+METRIC_DEFINE_gauge_int64(table,
+                          writable_ill_partitions,
+                          dsn::metric_unit::kPartitions,
+                          "The number of writable ill partitions, which means primary = 1 && "
+                          "primary + secondary >= mutation_2pc_min_replica_count && "
+                          "primary + secondary < max_replica_count");
+
+METRIC_DEFINE_gauge_int64(table,
+                          healthy_partitions,
+                          dsn::metric_unit::kPartitions,
+                          "The number of healthy partitions, which means primary = 1 && "
+                          "primary + secondary >= max_replica_count");
+
+METRIC_DEFINE_counter(table,
+                      partition_configuration_changes,
+                      dsn::metric_unit::kChanges,
+                      "The number of times the configuration has been changed");
+
+METRIC_DEFINE_counter(table,
+                      unwritable_partition_changes,
+                      dsn::metric_unit::kChanges,
+                      "The number of times the status of partition has been changed to unwritable");
+
+METRIC_DEFINE_counter(table,
+                      writable_partition_changes,
+                      dsn::metric_unit::kChanges,
+                      "The number of times the status of partition has been changed to writable");
+
+namespace dsn {
+
+namespace {
+
+metric_entity_ptr instantiate_table_metric_entity(int32_t table_id)
+{
+    auto entity_id = fmt::format("table_{}", table_id);
+
+    return METRIC_ENTITY_table.instantiate(entity_id, {{"table_id", std::to_string(table_id)}});
+}
+
+} // anonymous namespace
+
+table_metrics::table_metrics(int32_t table_id)
+    : _table_id(table_id),
+      _table_metric_entity(instantiate_table_metric_entity(table_id)),
+      METRIC_VAR_INIT_table(dead_partitions),
+      METRIC_VAR_INIT_table(unreadable_partitions),
+      METRIC_VAR_INIT_table(unwritable_partitions),
+      METRIC_VAR_INIT_table(writable_ill_partitions),
+      METRIC_VAR_INIT_table(healthy_partitions),
+      METRIC_VAR_INIT_table(partition_configuration_changes),
+      METRIC_VAR_INIT_table(unwritable_partition_changes),
+      METRIC_VAR_INIT_table(writable_partition_changes)
+{
+}
+
+const metric_entity_ptr &table_metrics::table_metric_entity() const
+{
+    CHECK_NOTNULL(_table_metric_entity,
+                  "table metric entity should has been instantiated: "
+                  "uninitialized entity cannot be used to instantiate "
+                  "metric");
+    return _table_metric_entity;
+}
+
+bool operator==(const table_metrics &lhs, const table_metrics &rhs)
+{
+    if (&lhs == &rhs) {
+        return true;
+    }
+
+    if (lhs.table_metric_entity().get() != rhs.table_metric_entity().get()) {
+        CHECK_NE(lhs.table_id(), rhs.table_id());
+        return false;
+    }
+
+    CHECK_EQ(lhs.table_id(), rhs.table_id());
+    return true;
+}
+
+bool operator!=(const table_metrics &lhs, const table_metrics &rhs) { return !(lhs == rhs); }
+
+void table_metric_entities::create_entity(int32_t table_id)
+{
+    utils::auto_write_lock l(_lock);
+
+    entity_map::const_iterator iter = _entities.find(table_id);
+    if (dsn_unlikely(iter != _entities.end())) {
+        return;
+    }
+
+    _entities[table_id] = std::make_unique<table_metrics>(table_id);
+}
+
+void table_metric_entities::remove_entity(int32_t table_id)
+{
+    utils::auto_write_lock l(_lock);
+
+    entity_map::const_iterator iter = _entities.find(table_id);
+    if (dsn_unlikely(iter == _entities.end())) {
+        return;
+    }
+
+    _entities.erase(iter);
+}
+
+void table_metric_entities::clear_entities()
+{
+    utils::auto_write_lock l(_lock);
+    _entities.clear();
+}
+
+bool operator==(const table_metric_entities &lhs, const table_metric_entities &rhs)
+{
+    if (&lhs == &rhs) {
+        return true;
+    }
+
+    utils::auto_read_lock l1(lhs._lock);
+    utils::auto_read_lock l2(rhs._lock);
+
+    if (lhs._entities.size() != rhs._entities.size()) {
+        return false;
+    }
+
+    for (const auto &lhs_entity : lhs._entities) {
+        auto rhs_entity = rhs._entities.find(lhs_entity.first);
+        if (rhs_entity == rhs._entities.end()) {
+            return false;
+        }
+
+        if (*(lhs_entity.second) != *(rhs_entity->second)) {
+            return false;
+        }
+    }
+
+    return true;
+}
+
+} // namespace dsn
diff --git a/src/meta/table_metrics.h b/src/meta/table_metrics.h
new file mode 100644
index 000000000..de5db3888
--- /dev/null
+++ b/src/meta/table_metrics.h
@@ -0,0 +1,136 @@
+// 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 <stdint.h>
+#include <memory>
+#include <unordered_map>
+#include <utility>
+
+#include "utils/autoref_ptr.h"
+#include "utils/metrics.h"
+#include "utils/ports.h"
+#include "utils/synchronize.h"
+
+namespace dsn {
+class table_metric_entities;
+
+// Maintain a table-level metric entity of meta, and all metrics attached to it.
+class table_metrics
+{
+public:
+    table_metrics(int32_t table_id);
+    ~table_metrics() = default;
+
+    inline int32_t table_id() const { return _table_id; }
+    const metric_entity_ptr &table_metric_entity() const;
+
+    METRIC_DEFINE_SET_METHOD(dead_partitions, int64_t)
+    METRIC_DEFINE_SET_METHOD(unreadable_partitions, int64_t)
+    METRIC_DEFINE_SET_METHOD(unwritable_partitions, int64_t)
+    METRIC_DEFINE_SET_METHOD(writable_ill_partitions, int64_t)
+    METRIC_DEFINE_SET_METHOD(healthy_partitions, int64_t)
+    METRIC_DEFINE_INCREMENT_METHOD(partition_configuration_changes)
+    METRIC_DEFINE_INCREMENT_METHOD(unwritable_partition_changes)
+    METRIC_DEFINE_INCREMENT_METHOD(writable_partition_changes)
+
+private:
+    const int32_t _table_id;
+
+    const metric_entity_ptr _table_metric_entity;
+    METRIC_VAR_DECLARE_gauge_int64(dead_partitions);
+    METRIC_VAR_DECLARE_gauge_int64(unreadable_partitions);
+    METRIC_VAR_DECLARE_gauge_int64(unwritable_partitions);
+    METRIC_VAR_DECLARE_gauge_int64(writable_ill_partitions);
+    METRIC_VAR_DECLARE_gauge_int64(healthy_partitions);
+    METRIC_VAR_DECLARE_counter(partition_configuration_changes);
+    METRIC_VAR_DECLARE_counter(unwritable_partition_changes);
+    METRIC_VAR_DECLARE_counter(writable_partition_changes);
+
+    DISALLOW_COPY_AND_ASSIGN(table_metrics);
+};
+
+bool operator==(const table_metrics &lhs, const table_metrics &rhs);
+bool operator!=(const table_metrics &lhs, const table_metrics &rhs);
+
+#define METRIC_DEFINE_TABLE_SET_METHOD(name, value_type)                                           \
+    void set_##name(int32_t table_id, value_type value)                                            \
+    {                                                                                              \
+        utils::auto_read_lock l(_lock);                                                            \
+                                                                                                   \
+        entity_map::const_iterator iter = _entities.find(table_id);                                \
+        if (dsn_unlikely(iter == _entities.end())) {                                               \
+            return;                                                                                \
+        }                                                                                          \
+        METRIC_CALL_SET_METHOD(*(iter->second), name, value);                                      \
+    }
+
+#define METRIC_CALL_TABLE_SET_METHOD(obj, name, table_id, value) (obj).set_##name(table_id, value)
+
+#define METRIC_DEFINE_TABLE_INCREMENT_METHOD(name)                                                 \
+    void increment_##name(int32_t table_id)                                                        \
+    {                                                                                              \
+        utils::auto_read_lock l(_lock);                                                            \
+                                                                                                   \
+        entity_map::const_iterator iter = _entities.find(table_id);                                \
+        if (dsn_unlikely(iter == _entities.end())) {                                               \
+            return;                                                                                \
+        }                                                                                          \
+        METRIC_CALL_INCREMENT_METHOD(*(iter->second), name);                                       \
+    }
+
+#define METRIC_CALL_TABLE_INCREMENT_METHOD(obj, name, table_id) (obj).increment_##name(table_id)
+
+// Manage the lifetime of all table-level metric entities of meta.
+//
+// To instantiate a new table-level entity, just call create_entity(). Once the entity instance
+// is not needed, just call remove_entity() (after `entity_retirement_delay_ms` milliseconds it
+// would be retired).
+class table_metric_entities
+{
+public:
+    using entity_map = std::unordered_map<int, std::unique_ptr<table_metrics>>;
+
+    table_metric_entities() = default;
+    ~table_metric_entities() = default;
+
+    void create_entity(int32_t table_id);
+    void remove_entity(int32_t table_id);
+    void clear_entities();
+
+    METRIC_DEFINE_TABLE_SET_METHOD(dead_partitions, int64_t)
+    METRIC_DEFINE_TABLE_SET_METHOD(unreadable_partitions, int64_t)
+    METRIC_DEFINE_TABLE_SET_METHOD(unwritable_partitions, int64_t)
+    METRIC_DEFINE_TABLE_SET_METHOD(writable_ill_partitions, int64_t)
+    METRIC_DEFINE_TABLE_SET_METHOD(healthy_partitions, int64_t)
+    METRIC_DEFINE_TABLE_INCREMENT_METHOD(partition_configuration_changes)
+    METRIC_DEFINE_TABLE_INCREMENT_METHOD(unwritable_partition_changes)
+    METRIC_DEFINE_TABLE_INCREMENT_METHOD(writable_partition_changes)
+
+private:
+    friend bool operator==(const table_metric_entities &, const table_metric_entities &);
+
+    mutable utils::rw_lock_nr _lock;
+    entity_map _entities;
+
+    DISALLOW_COPY_AND_ASSIGN(table_metric_entities);
+};
+
+bool operator==(const table_metric_entities &lhs, const table_metric_entities &rhs);
+
+} // namespace dsn
diff --git a/src/meta/test/state_sync_test.cpp b/src/meta/test/state_sync_test.cpp
index 2e1afacf8..657bd8157 100644
--- a/src/meta/test/state_sync_test.cpp
+++ b/src/meta/test/state_sync_test.cpp
@@ -248,6 +248,7 @@ void meta_service_test_app::state_sync_test()
         for (const auto &iter : ss1->_exist_apps) {
             ASSERT_TRUE(ss2->_exist_apps.find(iter.first) != ss2->_exist_apps.end());
         }
+        ASSERT_EQ(ss1->_table_metric_entities, ss2->_table_metric_entities);
 
         // then we dump the content to local file with binary format
         std::cerr << "test dump to local file from zookeeper's storage" << std::endl;
@@ -268,6 +269,7 @@ void meta_service_test_app::state_sync_test()
         for (const auto &iter : ss1->_exist_apps) {
             ASSERT_TRUE(ss2->_exist_apps.find(iter.first) != ss2->_exist_apps.end());
         }
+        ASSERT_EQ(ss1->_table_metric_entities, ss2->_table_metric_entities);
         ss2->initialize_node_state();
 
         // then let's test the query configuration calls
diff --git a/src/meta/test/update_configuration_test.cpp b/src/meta/test/update_configuration_test.cpp
index 4feba0923..cb30bff2b 100644
--- a/src/meta/test/update_configuration_test.cpp
+++ b/src/meta/test/update_configuration_test.cpp
@@ -52,6 +52,7 @@
 #include "meta/meta_service.h"
 #include "meta/partition_guardian.h"
 #include "meta/server_state.h"
+#include "meta/table_metrics.h"
 #include "meta/test/misc/misc.h"
 #include "meta_admin_types.h"
 #include "meta_service_test_app.h"
@@ -495,6 +496,7 @@ void meta_service_test_app::cannot_run_balancer_test()
     std::shared_ptr<app_state> the_app = app_state::create(info);
     svc->_state->_all_apps.emplace(info.app_id, the_app);
     svc->_state->_exist_apps.emplace(info.app_name, the_app);
+    svc->_state->_table_metric_entities.create_entity(info.app_id);
 
     dsn::partition_configuration &pc = the_app->partitions[0];
     pc.primary = nodes[0];
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index a230aa1f9..ba85c3fd4 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -166,6 +166,7 @@ class error_code;
 #define METRIC_VAR_INIT_replica(name, ...) METRIC_VAR_INIT(name, replica, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_server(name, ...) METRIC_VAR_INIT(name, server, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_disk(name, ...) METRIC_VAR_INIT(name, disk, ##__VA_ARGS__)
+#define METRIC_VAR_INIT_table(name, ...) METRIC_VAR_INIT(name, table, ##__VA_ARGS__)
 
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
@@ -198,7 +199,12 @@ class error_code;
 #define METRIC_DEFINE_SET_METHOD(name, value_type)                                                 \
     void set_##name(value_type value) { METRIC_VAR_SET(name, value); }
 
-#define METRIC_CALL_SET_METHOD(obj, name, value) obj.set_##name(value)
+#define METRIC_CALL_SET_METHOD(obj, name, value) (obj).set_##name(value)
+
+#define METRIC_DEFINE_INCREMENT_METHOD(name)                                                       \
+    void increment_##name() { METRIC_VAR_INCREMENT(name); }
+
+#define METRIC_CALL_INCREMENT_METHOD(obj, name) (obj).increment_##name()
 
 namespace dsn {
 class metric;                  // IWYU pragma: keep
@@ -621,6 +627,7 @@ enum class metric_unit : size_t
     kMegaBytes,
     kCapacityUnits,
     kPercent,
+    kPartitions,
     kRequests,
     kSeeks,
     kPointLookups,
@@ -631,6 +638,7 @@ enum class metric_unit : size_t
     kFlushes,
     kCompactions,
     kWrites,
+    kChanges,
     kInvalidUnit,
 };
 


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


[incubator-pegasus] 23/23: feat(new_metrics): migrate metrics for replica_stub (part 4) (#1463)

Posted by wa...@apache.org.
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 72f6e38367b4ff41dcdee5452c6dac28d2095433
Author: Dan Wang <wa...@apache.org>
AuthorDate: Fri May 5 10:22:39 2023 +0800

    feat(new_metrics): migrate metrics for replica_stub (part 4) (#1463)
    
    https://github.com/apache/incubator-pegasus/issues/1454
    
    This is the 4th part of migrating metrics of replica_stub to new framework.
    
    During this migration, there are 3 metrics which are changed from server-level
    to replica-level, including the number of confirmed/pending mutations for dup
    and the number of write requests whose size exceeds threshold.
    
    Another 5 metrics are still kept server-level, including the memory bytes that
    are released by tcmalloc recently, the number of failed read/write requests
    and the number of busy read/write requests.
---
 src/replica/duplication/duplication_sync_timer.cpp |   7 +-
 src/replica/duplication/replica_duplicator.cpp     |  15 ++--
 src/replica/duplication/replica_duplicator.h       |   6 ++
 .../duplication/replica_duplicator_manager.cpp     |  16 +++-
 .../duplication/replica_duplicator_manager.h       |  13 ++-
 src/replica/replica.cpp                            |  13 ++-
 src/replica/replica.h                              |   5 ++
 src/replica/replica_2pc.cpp                        |   4 +-
 src/replica/replica_stub.cpp                       | 100 ++++++++++-----------
 src/replica/replica_stub.h                         |  24 ++---
 src/replica/test/replica_test.cpp                  |  10 +--
 src/utils/metrics.h                                |   9 ++
 12 files changed, 125 insertions(+), 97 deletions(-)

diff --git a/src/replica/duplication/duplication_sync_timer.cpp b/src/replica/duplication/duplication_sync_timer.cpp
index d04d014fe..da1cd933f 100644
--- a/src/replica/duplication/duplication_sync_timer.cpp
+++ b/src/replica/duplication/duplication_sync_timer.cpp
@@ -26,8 +26,6 @@
 #include "common/replication.codes.h"
 #include "duplication_sync_timer.h"
 #include "metadata_types.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/replica.h"
 #include "replica/replica_stub.h"
 #include "replica_duplicator_manager.h"
@@ -37,6 +35,7 @@
 #include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/threadpool_code.h"
 
 namespace dsn {
@@ -65,15 +64,13 @@ void duplication_sync_timer::run()
     req->node = _stub->primary_address();
 
     // collects confirm points from all primaries on this server
-    uint64_t pending_muts_cnt = 0;
     for (const replica_ptr &r : get_all_primaries()) {
         auto confirmed = r->get_duplication_manager()->get_duplication_confirms_to_update();
         if (!confirmed.empty()) {
             req->confirm_list[r->get_gpid()] = std::move(confirmed);
         }
-        pending_muts_cnt += r->get_duplication_manager()->get_pending_mutations_count();
+        METRIC_SET(*r, dup_pending_mutations);
     }
-    _stub->_counter_dup_pending_mutations_count->set(pending_muts_cnt);
 
     duplication_sync_rpc rpc(std::move(req), RPC_CM_DUPLICATION_SYNC, 3_s);
     rpc_address meta_server_address(_stub->get_meta_server_address());
diff --git a/src/replica/duplication/replica_duplicator.cpp b/src/replica/duplication/replica_duplicator.cpp
index a19947fe2..e8018db34 100644
--- a/src/replica/duplication/replica_duplicator.cpp
+++ b/src/replica/duplication/replica_duplicator.cpp
@@ -32,15 +32,18 @@
 #include "dsn.layer2_types.h"
 #include "duplication_pipeline.h"
 #include "load_from_private_log.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/mutation_log.h"
 #include "replica/replica.h"
-#include "replica/replica_stub.h"
 #include "runtime/task/async_calls.h"
 #include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
+
+METRIC_DEFINE_counter(replica,
+                      dup_confirmed_mutations,
+                      dsn::metric_unit::kMutations,
+                      "The number of confirmed mutations for dup");
 
 namespace dsn {
 namespace replication {
@@ -50,7 +53,8 @@ replica_duplicator::replica_duplicator(const duplication_entry &ent, replica *r)
       _id(ent.dupid),
       _remote_cluster_name(ent.remote),
       _replica(r),
-      _stub(r->get_replica_stub())
+      _stub(r->get_replica_stub()),
+      METRIC_VAR_INIT_replica(dup_confirmed_mutations)
 {
     _status = ent.status;
 
@@ -222,7 +226,8 @@ error_s replica_duplicator::update_progress(const duplication_progress &p)
     }
     if (_progress.confirmed_decree > last_confirmed_decree) {
         // has confirmed_decree updated.
-        _stub->_counter_dup_confirmed_rate->add(_progress.confirmed_decree - last_confirmed_decree);
+        METRIC_VAR_INCREMENT_BY(dup_confirmed_mutations,
+                                _progress.confirmed_decree - last_confirmed_decree);
     }
 
     return error_s::ok();
diff --git a/src/replica/duplication/replica_duplicator.h b/src/replica/duplication/replica_duplicator.h
index 1b2526d29..fa185b8c7 100644
--- a/src/replica/duplication/replica_duplicator.h
+++ b/src/replica/duplication/replica_duplicator.h
@@ -29,6 +29,7 @@
 #include "runtime/pipeline.h"
 #include "runtime/task/task_tracker.h"
 #include "utils/errors.h"
+#include "utils/metrics.h"
 #include "utils/zlocks.h"
 
 namespace dsn {
@@ -169,6 +170,11 @@ private:
     std::unique_ptr<load_mutation> _load;
     std::unique_ptr<ship_mutation> _ship;
     std::unique_ptr<load_from_private_log> _load_private;
+
+    // <- Duplication Metrics ->
+    // TODO(wutao1): calculate the counters independently for each remote cluster
+    //               if we need to duplicate to multiple clusters someday.
+    METRIC_VAR_DECLARE_counter(dup_confirmed_mutations);
 };
 
 typedef std::unique_ptr<replica_duplicator> replica_duplicator_u_ptr;
diff --git a/src/replica/duplication/replica_duplicator_manager.cpp b/src/replica/duplication/replica_duplicator_manager.cpp
index 56ec34a39..ef53a9b43 100644
--- a/src/replica/duplication/replica_duplicator_manager.cpp
+++ b/src/replica/duplication/replica_duplicator_manager.cpp
@@ -23,12 +23,24 @@
 #include "common/gpid.h"
 #include "replica/duplication/replica_duplicator.h"
 #include "replica_duplicator_manager.h"
+#include "utils/autoref_ptr.h"
 #include "utils/errors.h"
 #include "utils/fmt_logging.h"
+#include "utils/string_view.h"
+
+METRIC_DEFINE_gauge_int64(replica,
+                          dup_pending_mutations,
+                          dsn::metric_unit::kMutations,
+                          "The number of pending mutations for dup");
 
 namespace dsn {
 namespace replication {
 
+replica_duplicator_manager::replica_duplicator_manager(replica *r)
+    : replica_base(r), _replica(r), METRIC_VAR_INIT_replica(dup_pending_mutations)
+{
+}
+
 std::vector<duplication_confirm_entry>
 replica_duplicator_manager::get_duplication_confirms_to_update() const
 {
@@ -149,13 +161,13 @@ void replica_duplicator_manager::update_confirmed_decree_if_secondary(decree con
     }
 }
 
-int64_t replica_duplicator_manager::get_pending_mutations_count() const
+void replica_duplicator_manager::METRIC_FUNC_NAME_SET(dup_pending_mutations)()
 {
     int64_t total = 0;
     for (const auto &dup : _duplications) {
         total += dup.second->get_pending_mutations_count();
     }
-    return total;
+    METRIC_VAR_SET(dup_pending_mutations, total);
 }
 
 std::vector<replica_duplicator_manager::dup_state>
diff --git a/src/replica/duplication/replica_duplicator_manager.h b/src/replica/duplication/replica_duplicator_manager.h
index 20bdcd0ac..f2b1593e4 100644
--- a/src/replica/duplication/replica_duplicator_manager.h
+++ b/src/replica/duplication/replica_duplicator_manager.h
@@ -31,6 +31,7 @@
 #include "replica/replica_base.h"
 #include "replica_duplicator.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/zlocks.h"
 
 namespace dsn {
@@ -43,7 +44,7 @@ namespace replication {
 class replica_duplicator_manager : public replica_base
 {
 public:
-    explicit replica_duplicator_manager(replica *r) : replica_base(r), _replica(r) {}
+    explicit replica_duplicator_manager(replica *r);
 
     // Immediately stop duplication in the following conditions:
     // - replica is not primary on replica-server perspective (status != PRIMARY)
@@ -77,9 +78,8 @@ public:
     /// \see replica_check.cpp
     void update_confirmed_decree_if_secondary(decree confirmed);
 
-    /// Sums up the number of pending mutations for all duplications
-    /// on this replica, for metric "dup.pending_mutations_count".
-    int64_t get_pending_mutations_count() const;
+    /// Sums up the number of pending mutations for all duplications on this replica.
+    void METRIC_FUNC_NAME_SET(dup_pending_mutations)();
 
     struct dup_state
     {
@@ -121,6 +121,11 @@ private:
     // avoid thread conflict between replica::on_checkpoint_timer and
     // duplication_sync_timer.
     mutable zlock _lock;
+
+    // <- Duplication Metrics ->
+    // TODO(wutao1): calculate the counters independently for each remote cluster
+    //               if we need to duplicate to multiple clusters someday.
+    METRIC_VAR_DECLARE_gauge_int64(dup_pending_mutations);
 };
 
 } // namespace replication
diff --git a/src/replica/replica.cpp b/src/replica/replica.cpp
index a660ab509..a4d2068b2 100644
--- a/src/replica/replica.cpp
+++ b/src/replica/replica.cpp
@@ -204,6 +204,11 @@ METRIC_DEFINE_counter(replica,
                       dsn::metric_unit::kCheckpoints,
                       "The number of triggered emergency checkpoints");
 
+METRIC_DEFINE_counter(replica,
+                      write_size_exceed_threshold_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of write requests whose size exceeds threshold");
+
 namespace dsn {
 namespace replication {
 
@@ -282,7 +287,8 @@ replica::replica(replica_stub *stub,
       METRIC_VAR_INIT_replica(learn_successful_count),
       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(emergency_checkpoints),
+      METRIC_VAR_INIT_replica(write_size_exceed_threshold_requests)
 {
     CHECK(!_app_info.app_type.empty(), "");
     CHECK_NOTNULL(stub, "");
@@ -718,5 +724,10 @@ bool replica::access_controller_allowed(message_ex *msg, const ranger::access_ty
 
 int64_t replica::get_backup_request_count() const { return METRIC_VAR_VALUE(backup_requests); }
 
+void replica::METRIC_FUNC_NAME_SET(dup_pending_mutations)()
+{
+    METRIC_SET(*_duplication_mgr, dup_pending_mutations);
+}
+
 } // namespace replication
 } // namespace dsn
diff --git a/src/replica/replica.h b/src/replica/replica.h
index 8feefb109..dcaf813e9 100644
--- a/src/replica/replica.h
+++ b/src/replica/replica.h
@@ -288,6 +288,9 @@ public:
     disk_status::type get_disk_status() { return _disk_status; }
     std::string get_replica_disk_tag() const { return _disk_tag; }
 
+    METRIC_DEFINE_VALUE(write_size_exceed_threshold_requests, int64_t)
+    void METRIC_FUNC_NAME_SET(dup_pending_mutations)();
+
     static const std::string kAppInfo;
 
 protected:
@@ -682,6 +685,8 @@ private:
 
     METRIC_VAR_DECLARE_counter(emergency_checkpoints);
 
+    METRIC_VAR_DECLARE_counter(write_size_exceed_threshold_requests);
+
     dsn::task_tracker _tracker;
     // the thread access checker
     dsn::thread_access_checker _checker;
diff --git a/src/replica/replica_2pc.cpp b/src/replica/replica_2pc.cpp
index 970df6688..50dcd4b46 100644
--- a/src/replica/replica_2pc.cpp
+++ b/src/replica/replica_2pc.cpp
@@ -47,8 +47,6 @@
 #include "metadata_types.h"
 #include "mutation.h"
 #include "mutation_log.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica.h"
 #include "replica/prepare_list.h"
 #include "replica/replica_context.h"
@@ -148,7 +146,7 @@ void replica::on_client_write(dsn::message_ex *request, bool ignore_throttling)
             request_info,
             request->body_size(),
             FLAGS_max_allowed_write_size);
-        _stub->_counter_recent_write_size_exceed_threshold_count->increment();
+        METRIC_VAR_INCREMENT(write_size_exceed_threshold_requests);
         response_client_write(request, ERR_INVALID_DATA);
         return;
     }
diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp
index 48718952c..3c7a1fc4d 100644
--- a/src/replica/replica_stub.cpp
+++ b/src/replica/replica_stub.cpp
@@ -164,6 +164,33 @@ METRIC_DEFINE_gauge_int64(server,
                           dsn::metric_unit::kDirs,
                           "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");
+#endif
+
+METRIC_DEFINE_counter(server,
+                      read_failed_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of failed read requests");
+
+METRIC_DEFINE_counter(server,
+                      write_failed_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of failed write requests");
+
+METRIC_DEFINE_counter(server,
+                      read_busy_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of busy read requests");
+
+METRIC_DEFINE_counter(server,
+                      write_busy_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of busy write requests");
+
 namespace dsn {
 namespace replication {
 
@@ -279,7 +306,14 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/,
       METRIC_VAR_INIT_server(replica_error_dirs),
       METRIC_VAR_INIT_server(replica_garbage_dirs),
       METRIC_VAR_INIT_server(replica_tmp_dirs),
-      METRIC_VAR_INIT_server(replica_origin_dirs)
+      METRIC_VAR_INIT_server(replica_origin_dirs),
+#ifdef DSN_ENABLE_GPERF
+      METRIC_VAR_INIT_server(tcmalloc_released_bytes),
+#endif
+      METRIC_VAR_INIT_server(read_failed_requests),
+      METRIC_VAR_INIT_server(write_failed_requests),
+      METRIC_VAR_INIT_server(read_busy_requests),
+      METRIC_VAR_INIT_server(write_busy_requests)
 {
 #ifdef DSN_ENABLE_GPERF
     _is_releasing_memory = false;
@@ -297,18 +331,6 @@ replica_stub::~replica_stub(void) { close(); }
 
 void replica_stub::install_perf_counters()
 {
-    // <- Duplication Metrics ->
-
-    _counter_dup_confirmed_rate.init_app_counter("eon.replica_stub",
-                                                 "dup.confirmed_rate",
-                                                 COUNTER_TYPE_RATE,
-                                                 "increasing rate of confirmed mutations");
-    _counter_dup_pending_mutations_count.init_app_counter(
-        "eon.replica_stub",
-        "dup.pending_mutations_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "number of mutations pending for duplication");
-
     // <- Cold Backup Metrics ->
 
     _counter_cold_backup_running_count.init_app_counter("eon.replica_stub",
@@ -366,29 +388,6 @@ void replica_stub::install_perf_counters()
         COUNTER_TYPE_NUMBER,
         "current cold backup max upload file size");
 
-    _counter_recent_read_fail_count.init_app_counter("eon.replica_stub",
-                                                     "recent.read.fail.count",
-                                                     COUNTER_TYPE_VOLATILE_NUMBER,
-                                                     "read fail count in the recent period");
-    _counter_recent_write_fail_count.init_app_counter("eon.replica_stub",
-                                                      "recent.write.fail.count",
-                                                      COUNTER_TYPE_VOLATILE_NUMBER,
-                                                      "write fail count in the recent period");
-    _counter_recent_read_busy_count.init_app_counter("eon.replica_stub",
-                                                     "recent.read.busy.count",
-                                                     COUNTER_TYPE_VOLATILE_NUMBER,
-                                                     "read busy count in the recent period");
-    _counter_recent_write_busy_count.init_app_counter("eon.replica_stub",
-                                                      "recent.write.busy.count",
-                                                      COUNTER_TYPE_VOLATILE_NUMBER,
-                                                      "write busy count in the recent period");
-
-    _counter_recent_write_size_exceed_threshold_count.init_app_counter(
-        "eon.replica_stub",
-        "recent_write_size_exceed_threshold_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "write size exceed threshold count in the recent period");
-
     // <- Bulk Load Metrics ->
 
     _counter_bulk_load_running_count.init_app_counter("eon.replica_stub",
@@ -435,13 +434,6 @@ void replica_stub::install_perf_counters()
                                                              COUNTER_TYPE_NUMBER,
                                                              "bulk load max duration time(ms)");
 
-#ifdef DSN_ENABLE_GPERF
-    _counter_tcmalloc_release_memory_size.init_app_counter("eon.replica_stub",
-                                                           "tcmalloc.release.memory.size",
-                                                           COUNTER_TYPE_NUMBER,
-                                                           "current tcmalloc release memory size");
-#endif
-
     // <- Partition split Metrics ->
 
     _counter_replicas_splitting_count.init_app_counter("eon.replica_stub",
@@ -1662,15 +1654,17 @@ void replica_stub::response_client(gpid id,
                                    error_code error)
 {
     if (error == ERR_BUSY) {
-        if (is_read)
-            _counter_recent_read_busy_count->increment();
-        else
-            _counter_recent_write_busy_count->increment();
+        if (is_read) {
+            METRIC_VAR_INCREMENT(read_busy_requests);
+        } else {
+            METRIC_VAR_INCREMENT(write_busy_requests);
+        }
     } else if (error != ERR_OK) {
-        if (is_read)
-            _counter_recent_read_fail_count->increment();
-        else
-            _counter_recent_write_fail_count->increment();
+        if (is_read) {
+            METRIC_VAR_INCREMENT(read_failed_requests);
+        } else {
+            METRIC_VAR_INCREMENT(write_failed_requests);
+        }
         LOG_ERROR("{}@{}: {} fail: client = {}, code = {}, timeout = {}, status = {}, error = {}",
                   id,
                   _primary_address_str,
@@ -2870,7 +2864,7 @@ uint64_t replica_stub::gc_tcmalloc_memory(bool release_all)
     auto tcmalloc_released_bytes = 0;
     if (!_release_tcmalloc_memory) {
         _is_releasing_memory.store(false);
-        _counter_tcmalloc_release_memory_size->set(tcmalloc_released_bytes);
+        METRIC_VAR_SET(tcmalloc_released_bytes, tcmalloc_released_bytes);
         return tcmalloc_released_bytes;
     }
 
@@ -2901,7 +2895,7 @@ uint64_t replica_stub::gc_tcmalloc_memory(bool release_all)
             release_bytes -= 1024 * 1024;
         }
     }
-    _counter_tcmalloc_release_memory_size->set(tcmalloc_released_bytes);
+    METRIC_VAR_SET(tcmalloc_released_bytes, tcmalloc_released_bytes);
     _is_releasing_memory.store(false);
     return tcmalloc_released_bytes;
 }
diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h
index 7bcf4e68b..e6243da78 100644
--- a/src/replica/replica_stub.h
+++ b/src/replica/replica_stub.h
@@ -521,11 +521,14 @@ private:
     METRIC_VAR_DECLARE_gauge_int64(replica_tmp_dirs);
     METRIC_VAR_DECLARE_gauge_int64(replica_origin_dirs);
 
-    // <- Duplication Metrics ->
-    // TODO(wutao1): calculate the counters independently for each remote cluster
-    //               if we need to duplicate to multiple clusters someday.
-    perf_counter_wrapper _counter_dup_confirmed_rate;
-    perf_counter_wrapper _counter_dup_pending_mutations_count;
+#ifdef DSN_ENABLE_GPERF
+    METRIC_VAR_DECLARE_gauge_int64(tcmalloc_released_bytes);
+#endif
+
+    METRIC_VAR_DECLARE_counter(read_failed_requests);
+    METRIC_VAR_DECLARE_counter(write_failed_requests);
+    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;
@@ -539,17 +542,6 @@ private:
     perf_counter_wrapper _counter_cold_backup_max_duration_time_ms;
     perf_counter_wrapper _counter_cold_backup_max_upload_file_size;
 
-    perf_counter_wrapper _counter_recent_read_fail_count;
-    perf_counter_wrapper _counter_recent_write_fail_count;
-    perf_counter_wrapper _counter_recent_read_busy_count;
-    perf_counter_wrapper _counter_recent_write_busy_count;
-
-    perf_counter_wrapper _counter_recent_write_size_exceed_threshold_count;
-
-#ifdef DSN_ENABLE_GPERF
-    perf_counter_wrapper _counter_tcmalloc_release_memory_size;
-#endif
-
     // <- Bulk load Metrics ->
     perf_counter_wrapper _counter_bulk_load_running_count;
     perf_counter_wrapper _counter_bulk_load_downloading_count;
diff --git a/src/replica/test/replica_test.cpp b/src/replica/test/replica_test.cpp
index 695417c86..9eb67eed5 100644
--- a/src/replica/test/replica_test.cpp
+++ b/src/replica/test/replica_test.cpp
@@ -38,8 +38,6 @@
 #include "dsn.layer2_types.h"
 #include "http/http_server.h"
 #include "metadata_types.h"
-#include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "replica/disk_cleaner.h"
 #include "replica/replica.h"
 #include "replica/replica_http_service.h"
@@ -60,6 +58,7 @@
 #include "utils/filesystem.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/string_conv.h"
 
 namespace dsn {
@@ -91,11 +90,6 @@ public:
         FLAGS_cold_backup_root = "test_cluster";
     }
 
-    int get_write_size_exceed_threshold_count()
-    {
-        return stub->_counter_recent_write_size_exceed_threshold_count->get_value();
-    }
-
     int64_t get_backup_request_count() const { return _mock_replica->get_backup_request_count(); }
 
     bool get_validate_partition_hash() const { return _mock_replica->_validate_partition_hash; }
@@ -274,7 +268,7 @@ TEST_F(replica_test, write_size_limited)
         stub->on_client_write(pid, write_request);
     }
 
-    ASSERT_EQ(get_write_size_exceed_threshold_count(), count);
+    ASSERT_EQ(count, METRIC_VALUE(*_mock_replica, write_size_exceed_threshold_requests));
 }
 
 TEST_F(replica_test, backup_request_count)
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index ec2fb6977..e90d5d9af 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -231,6 +231,14 @@ class error_code;
 // To be adaptive to self-defined `set` methods, arguments are declared as variadic.
 #define METRIC_SET(obj, name, ...) (obj).METRIC_FUNC_NAME_SET(name)(__VA_ARGS__)
 
+#define METRIC_FUNC_NAME_VALUE(name) get_##name
+
+#define METRIC_DEFINE_VALUE(name, value_type)                                                      \
+    value_type METRIC_FUNC_NAME_VALUE(name)() { return METRIC_VAR_VALUE(name); }
+
+// To be adaptive to self-defined `value` methods, arguments are declared as variadic.
+#define METRIC_VALUE(obj, name, ...) (obj).METRIC_FUNC_NAME_VALUE(name)(__VA_ARGS__)
+
 namespace dsn {
 class metric;                  // IWYU pragma: keep
 class metric_entity_prototype; // IWYU pragma: keep
@@ -667,6 +675,7 @@ enum class metric_unit : size_t
     kCheckpoints,
     kFlushes,
     kCompactions,
+    kMutations,
     kWrites,
     kChanges,
     kOperations,


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


[incubator-pegasus] 17/23: feat(new_metrics): migrate partition-level metrics for partition_guardian (#1440)

Posted by wa...@apache.org.
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 af3cad27d26ae79314c4216252e3512712068d5c
Author: Dan Wang <wa...@apache.org>
AuthorDate: Sun Apr 16 21:39:00 2023 +0800

    feat(new_metrics): migrate partition-level metrics for partition_guardian (#1440)
    
    https://github.com/apache/incubator-pegasus/issues/1331
    
    In perf counters, there's only one metric for partition_guardian, namely
    the number of operations that fail to choose the primary replica, which
    is server-level. It would be changed to partition-level in new metrics
    since this could give which partitions fail to choose primaries and how
    frequency those happen. Still, to compute table-level or server-level
    metrics just aggregate on partition-level ones.
---
 src/meta/partition_guardian.cpp | 14 ++++++--------
 src/meta/partition_guardian.h   |  2 --
 src/meta/table_metrics.cpp      |  8 +++++++-
 src/meta/table_metrics.h        |  5 +++++
 4 files changed, 18 insertions(+), 11 deletions(-)

diff --git a/src/meta/partition_guardian.cpp b/src/meta/partition_guardian.cpp
index e82ad8386..325508406 100644
--- a/src/meta/partition_guardian.cpp
+++ b/src/meta/partition_guardian.cpp
@@ -29,9 +29,11 @@
 #include "meta/meta_data.h"
 #include "meta/meta_service.h"
 #include "meta/server_load_balancer.h"
-#include "perf_counter/perf_counter.h"
+#include "meta/server_state.h"
+#include "meta/table_metrics.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/string_conv.h"
 #include "utils/strings.h"
 #include "utils/time_utils.h"
@@ -53,12 +55,6 @@ partition_guardian::partition_guardian(meta_service *svc) : _svc(svc)
     } else {
         _replica_assign_delay_ms_for_dropouts = 0;
     }
-
-    _recent_choose_primary_fail_count.init_app_counter(
-        "eon.server_load_balancer",
-        "recent_choose_primary_fail_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "choose primary fail count in the recent period");
 }
 
 pc_status partition_guardian::cure(meta_view view,
@@ -452,7 +448,9 @@ pc_status partition_guardian::on_missing_primary(meta_view &view, const dsn::gpi
             LOG_WARNING("{}: don't select any node for security reason, administrator can select "
                         "a proper one by shell",
                         gpid_name);
-            _recent_choose_primary_fail_count->increment();
+            METRIC_INCREMENT(_svc->get_server_state()->get_table_metric_entities(),
+                             choose_primary_failed_operations,
+                             gpid);
             ddd_partition_info pinfo;
             pinfo.config = pc;
             for (int i = 0; i < cc.dropped.size(); ++i) {
diff --git a/src/meta/partition_guardian.h b/src/meta/partition_guardian.h
index 23d78563a..9c77da7e5 100644
--- a/src/meta/partition_guardian.h
+++ b/src/meta/partition_guardian.h
@@ -29,7 +29,6 @@
 #include "dsn.layer2_types.h"
 #include "meta_admin_types.h"
 #include "meta_data.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "runtime/rpc/rpc_address.h"
 #include "utils/command_manager.h"
 #include "utils/zlocks.h"
@@ -91,7 +90,6 @@ private:
     }
 
     meta_service *_svc;
-    perf_counter_wrapper _recent_choose_primary_fail_count;
 
     mutable zlock _ddd_partitions_lock; // [
     std::map<gpid, ddd_partition_info> _ddd_partitions;
diff --git a/src/meta/table_metrics.cpp b/src/meta/table_metrics.cpp
index 73b992062..c63dd1f09 100644
--- a/src/meta/table_metrics.cpp
+++ b/src/meta/table_metrics.cpp
@@ -65,6 +65,11 @@ METRIC_DEFINE_counter(partition,
                       dsn::metric_unit::kOperations,
                       "The number of balance operations by greedy balancer that copy secondaries");
 
+METRIC_DEFINE_counter(partition,
+                      choose_primary_failed_operations,
+                      dsn::metric_unit::kOperations,
+                      "The number of operations that fail to choose the primary replica");
+
 METRIC_DEFINE_entity(table);
 
 // The number of partitions in each status, see `health_status` and `partition_health_status()`
@@ -133,7 +138,8 @@ partition_metrics::partition_metrics(int32_t table_id, int32_t partition_id)
       METRIC_VAR_INIT_partition(greedy_recent_balance_operations),
       METRIC_VAR_INIT_partition(greedy_move_primary_operations),
       METRIC_VAR_INIT_partition(greedy_copy_primary_operations),
-      METRIC_VAR_INIT_partition(greedy_copy_secondary_operations)
+      METRIC_VAR_INIT_partition(greedy_copy_secondary_operations),
+      METRIC_VAR_INIT_partition(choose_primary_failed_operations)
 {
 }
 
diff --git a/src/meta/table_metrics.h b/src/meta/table_metrics.h
index 39fec611d..7dc5e83a3 100644
--- a/src/meta/table_metrics.h
+++ b/src/meta/table_metrics.h
@@ -52,6 +52,8 @@ public:
     METRIC_DEFINE_INCREMENT_BY(greedy_copy_primary_operations)
     METRIC_DEFINE_INCREMENT_BY(greedy_copy_secondary_operations)
 
+    METRIC_DEFINE_INCREMENT(choose_primary_failed_operations)
+
 private:
     const int32_t _table_id;
     const int32_t _partition_id;
@@ -64,6 +66,7 @@ private:
     METRIC_VAR_DECLARE_counter(greedy_move_primary_operations);
     METRIC_VAR_DECLARE_counter(greedy_copy_primary_operations);
     METRIC_VAR_DECLARE_counter(greedy_copy_secondary_operations);
+    METRIC_VAR_DECLARE_counter(choose_primary_failed_operations);
 
     DISALLOW_COPY_AND_ASSIGN(partition_metrics);
 };
@@ -112,6 +115,7 @@ public:
     __METRIC_DEFINE_INCREMENT(partition_configuration_changes)
     __METRIC_DEFINE_INCREMENT(unwritable_partition_changes)
     __METRIC_DEFINE_INCREMENT(writable_partition_changes)
+    __METRIC_DEFINE_INCREMENT(choose_primary_failed_operations)
 
 #undef __METRIC_DEFINE_INCREMENT
 
@@ -221,6 +225,7 @@ public:
     __METRIC_DEFINE_INCREMENT(partition_configuration_changes)
     __METRIC_DEFINE_INCREMENT(unwritable_partition_changes)
     __METRIC_DEFINE_INCREMENT(writable_partition_changes)
+    __METRIC_DEFINE_INCREMENT(choose_primary_failed_operations)
 
 #undef __METRIC_DEFINE_INCREMENT
 


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


[incubator-pegasus] 19/23: feat(new_metrics): migrate metrics for replica_stub (part 1) (#1455)

Posted by wa...@apache.org.
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 eccc3be60133c31ad2e497b4b3609e3fbbacf913
Author: Dan Wang <wa...@apache.org>
AuthorDate: Wed Apr 19 22:57:26 2023 +0800

    feat(new_metrics): migrate metrics for replica_stub (part 1) (#1455)
    
    https://github.com/apache/incubator-pegasus/issues/1454
    
    This is the 1st part of migrating metrics of `replica_stub` to new framework.
    After migrating to new framework, the 3 metrics, including the total number
    of replicas, the number of opening/closing replicas, are still kept server-level.
    Another metric, the number of committed requests, is changed to replica-level.
    
    The naming of metric variable would lead to duplication with class member
    (such as `_opening_replicas` in `replica_stub` class). Therefore, a macro
    `METRIC_VAR_NAME` is introduced to manage the new naming, which is prefixed with
    `_metric_` to avoid duplication. Also, generated metric function names are also
    managed by related macros.
---
 src/meta/table_metrics.h             | 10 +++---
 src/replica/replica.cpp              | 10 ------
 src/replica/replica.h                |  5 ---
 src/replica/replica_stub.cpp         | 60 +++++++++++++++++++-----------------
 src/replica/replica_stub.h           |  8 ++---
 src/replica/replication_app_base.cpp | 10 ++++--
 src/replica/replication_app_base.h   |  4 +++
 src/utils/metrics.h                  | 40 ++++++++++++++----------
 src/utils/test/metrics_test.cpp      | 14 ++++-----
 9 files changed, 84 insertions(+), 77 deletions(-)

diff --git a/src/meta/table_metrics.h b/src/meta/table_metrics.h
index 7dc5e83a3..cec35e64b 100644
--- a/src/meta/table_metrics.h
+++ b/src/meta/table_metrics.h
@@ -93,7 +93,7 @@ public:
     METRIC_DEFINE_SET(healthy_partitions, int64_t)
 
 #define __METRIC_DEFINE_INCREMENT_BY(name)                                                         \
-    void increment_##name##_by(int32_t partition_id, int64_t x)                                    \
+    void METRIC_FUNC_NAME_INCREMENT_BY(name)(int32_t partition_id, int64_t x)                      \
     {                                                                                              \
         CHECK_LT(partition_id, _partition_metrics.size());                                         \
         METRIC_INCREMENT_BY(*(_partition_metrics[partition_id]), name, x);                         \
@@ -106,7 +106,7 @@ public:
 #undef __METRIC_DEFINE_INCREMENT_BY
 
 #define __METRIC_DEFINE_INCREMENT(name)                                                            \
-    void increment_##name(int32_t partition_id)                                                    \
+    void METRIC_FUNC_NAME_INCREMENT(name)(int32_t partition_id)                                    \
     {                                                                                              \
         CHECK_LT(partition_id, _partition_metrics.size());                                         \
         METRIC_INCREMENT(*(_partition_metrics[partition_id]), name);                               \
@@ -120,7 +120,7 @@ public:
 #undef __METRIC_DEFINE_INCREMENT
 
 #define __METRIC_DEFINE_SET(name, value_type)                                                      \
-    void set_##name(int32_t partition_id, value_type value)                                        \
+    void METRIC_FUNC_NAME_SET(name)(int32_t partition_id, value_type value)                        \
     {                                                                                              \
         CHECK_LT(partition_id, _partition_metrics.size());                                         \
         METRIC_SET(*(_partition_metrics[partition_id]), name, value);                              \
@@ -167,7 +167,7 @@ public:
     using partition_map = std::unordered_map<gpid, partition_stats>;
 
 #define __METRIC_DEFINE_INCREMENT(name)                                                            \
-    void increment_##name(const gpid &id, bool balance_checker)                                    \
+    void METRIC_FUNC_NAME_INCREMENT(name)(const gpid &id, bool balance_checker)                    \
     {                                                                                              \
         auto &partition = _partition_map[id];                                                      \
         ++(partition.greedy_recent_balance_operations);                                            \
@@ -210,7 +210,7 @@ public:
     void clear_entities();
 
 #define __METRIC_DEFINE_INCREMENT(name)                                                            \
-    void increment_##name(const gpid &id)                                                          \
+    void METRIC_FUNC_NAME_INCREMENT(name)(const gpid &id)                                          \
     {                                                                                              \
         utils::auto_read_lock l(_lock);                                                            \
                                                                                                    \
diff --git a/src/replica/replica.cpp b/src/replica/replica.cpp
index 8b8f22470..392c05300 100644
--- a/src/replica/replica.cpp
+++ b/src/replica/replica.cpp
@@ -48,7 +48,6 @@
 #include "mutation.h"
 #include "mutation_log.h"
 #include "perf_counter/perf_counter.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "perf_counter/perf_counters.h"
 #include "replica/prepare_list.h"
 #include "replica/replica_context.h"
@@ -227,15 +226,6 @@ void replica::update_last_checkpoint_generate_time()
         _last_checkpoint_generate_time_ms + rand::next_u64(max_interval_ms / 2, max_interval_ms);
 }
 
-//            //
-// Statistics //
-//            //
-
-void replica::update_commit_qps(int count)
-{
-    _stub->_counter_replicas_commit_qps->add((uint64_t)count);
-}
-
 void replica::init_state()
 {
     _inactive_is_transient = false;
diff --git a/src/replica/replica.h b/src/replica/replica.h
index 104854e20..ab9405895 100644
--- a/src/replica/replica.h
+++ b/src/replica/replica.h
@@ -280,11 +280,6 @@ public:
 
     replica_follower *get_replica_follower() const { return _replica_follower.get(); };
 
-    //
-    // Statistics
-    //
-    void update_commit_qps(int count);
-
     // routine for get extra envs from replica
     const std::map<std::string, std::string> &get_replica_extra_envs() const { return _extra_envs; }
 
diff --git a/src/replica/replica_stub.cpp b/src/replica/replica_stub.cpp
index 41b7e46a8..292eda470 100644
--- a/src/replica/replica_stub.cpp
+++ b/src/replica/replica_stub.cpp
@@ -98,6 +98,21 @@
 #include "remote_cmd/remote_command.h"
 #include "utils/fail_point.h"
 
+METRIC_DEFINE_gauge_int64(server,
+                          total_replicas,
+                          dsn::metric_unit::kReplicas,
+                          "The total number of replicas");
+
+METRIC_DEFINE_gauge_int64(server,
+                          opening_replicas,
+                          dsn::metric_unit::kReplicas,
+                          "The number of opening replicas");
+
+METRIC_DEFINE_gauge_int64(server,
+                          closing_replicas,
+                          dsn::metric_unit::kReplicas,
+                          "The number of closing replicas");
+
 namespace dsn {
 namespace replication {
 
@@ -200,7 +215,10 @@ replica_stub::replica_stub(replica_state_subscriber subscriber /*= nullptr*/,
       _learn_app_concurrent_count(0),
       _bulk_load_downloading_count(0),
       _manual_emergency_checkpointing_count(0),
-      _is_running(false)
+      _is_running(false),
+      METRIC_VAR_INIT_server(total_replicas),
+      METRIC_VAR_INIT_server(opening_replicas),
+      METRIC_VAR_INIT_server(closing_replicas)
 {
 #ifdef DSN_ENABLE_GPERF
     _is_releasing_memory = false;
@@ -218,20 +236,6 @@ replica_stub::~replica_stub(void) { close(); }
 
 void replica_stub::install_perf_counters()
 {
-    _counter_replicas_count.init_app_counter(
-        "eon.replica_stub", "replica(Count)", COUNTER_TYPE_NUMBER, "# in replica_stub._replicas");
-    _counter_replicas_opening_count.init_app_counter("eon.replica_stub",
-                                                     "opening.replica(Count)",
-                                                     COUNTER_TYPE_NUMBER,
-                                                     "# in replica_stub._opening_replicas");
-    _counter_replicas_closing_count.init_app_counter("eon.replica_stub",
-                                                     "closing.replica(Count)",
-                                                     COUNTER_TYPE_NUMBER,
-                                                     "# in replica_stub._closing_replicas");
-    _counter_replicas_commit_qps.init_app_counter("eon.replica_stub",
-                                                  "replicas.commit.qps",
-                                                  COUNTER_TYPE_RATE,
-                                                  "server-level commit throughput");
     _counter_replicas_learning_count.init_app_counter("eon.replica_stub",
                                                       "replicas.learning.count",
                                                       COUNTER_TYPE_NUMBER,
@@ -800,7 +804,7 @@ void replica_stub::initialize(const replication_options &opts, bool clear /* = f
 
     // attach rps
     _replicas = std::move(rps);
-    _counter_replicas_count->add((uint64_t)_replicas.size());
+    METRIC_VAR_INCREMENT_BY(total_replicas, _replicas.size());
     for (const auto &kv : _replicas) {
         _fs_manager.add_replica(kv.first, kv.second->dir());
     }
@@ -2057,10 +2061,10 @@ task_ptr replica_stub::begin_open_replica(
         if (rep->status() == partition_status::PS_INACTIVE && tsk->cancel(false)) {
             // reopen it
             _closing_replicas.erase(it);
-            _counter_replicas_closing_count->decrement();
+            METRIC_VAR_DECREMENT(closing_replicas);
 
             _replicas.emplace(id, rep);
-            _counter_replicas_count->increment();
+            METRIC_VAR_INCREMENT(total_replicas);
 
             _closed_replicas.erase(id);
 
@@ -2086,7 +2090,7 @@ task_ptr replica_stub::begin_open_replica(
         std::bind(&replica_stub::open_replica, this, app, id, group_check, configuration_update));
 
     _opening_replicas[id] = task;
-    _counter_replicas_opening_count->increment();
+    METRIC_VAR_INCREMENT(opening_replicas);
     _closed_replicas.erase(id);
 
     _replicas_lock.unlock_write();
@@ -2194,7 +2198,7 @@ void replica_stub::open_replica(
                      0,
                      "replica {} is not in _opening_replicas",
                      id.to_string());
-        _counter_replicas_opening_count->decrement();
+        METRIC_VAR_DECREMENT(opening_replicas);
         return;
     }
 
@@ -2204,13 +2208,13 @@ void replica_stub::open_replica(
                      0,
                      "replica {} is not in _opening_replicas",
                      id.to_string());
-        _counter_replicas_opening_count->decrement();
+        METRIC_VAR_DECREMENT(opening_replicas);
 
         CHECK(_replicas.find(id) == _replicas.end(),
               "replica {} is already in _replicas",
               id.to_string());
         _replicas.insert(replicas::value_type(rep->get_gpid(), rep));
-        _counter_replicas_count->increment();
+        METRIC_VAR_INCREMENT(total_replicas);
 
         _closed_replicas.erase(id);
     }
@@ -2367,7 +2371,7 @@ task_ptr replica_stub::begin_close_replica(replica_ptr r)
         return nullptr;
     }
 
-    _counter_replicas_count->decrement();
+    METRIC_VAR_DECREMENT(total_replicas);
 
     int delay_ms = 0;
     if (r->status() == partition_status::PS_INACTIVE) {
@@ -2386,7 +2390,7 @@ task_ptr replica_stub::begin_close_replica(replica_ptr r)
                                      0,
                                      std::chrono::milliseconds(delay_ms));
     _closing_replicas[id] = std::make_tuple(task, r, std::move(a_info), std::move(r_info));
-    _counter_replicas_closing_count->increment();
+    METRIC_VAR_INCREMENT(closing_replicas);
     return task;
 }
 
@@ -2406,7 +2410,7 @@ void replica_stub::close_replica(replica_ptr r)
         _closed_replicas.emplace(
             id, std::make_pair(std::get<2>(find->second), std::get<3>(find->second)));
         _closing_replicas.erase(find);
-        _counter_replicas_closing_count->decrement();
+        METRIC_VAR_DECREMENT(closing_replicas);
     }
 
     if (r->is_data_corrupted()) {
@@ -2870,7 +2874,7 @@ void replica_stub::close()
 
             task->cancel(true);
 
-            _counter_replicas_opening_count->decrement();
+            METRIC_VAR_DECREMENT(opening_replicas);
             _replicas_lock.lock_write();
             _opening_replicas.erase(_opening_replicas.begin());
         }
@@ -2878,7 +2882,7 @@ void replica_stub::close()
         while (!_replicas.empty()) {
             _replicas.begin()->second->close();
 
-            _counter_replicas_count->decrement();
+            METRIC_VAR_DECREMENT(total_replicas);
             _replicas.erase(_replicas.begin());
         }
     }
@@ -3037,7 +3041,7 @@ replica_ptr replica_stub::create_child_replica_if_not_found(gpid child_pid,
             if (rep != nullptr) {
                 auto pr = _replicas.insert(replicas::value_type(child_pid, rep));
                 CHECK(pr.second, "child replica {} has been existed", rep->name());
-                _counter_replicas_count->increment();
+                METRIC_VAR_INCREMENT(total_replicas);
                 _closed_replicas.erase(child_pid);
             }
             return rep;
diff --git a/src/replica/replica_stub.h b/src/replica/replica_stub.h
index 12f17a1cf..21e68c843 100644
--- a/src/replica/replica_stub.h
+++ b/src/replica/replica_stub.h
@@ -71,6 +71,7 @@
 #include "utils/autoref_ptr.h"
 #include "utils/error_code.h"
 #include "utils/flags.h"
+#include "utils/metrics.h"
 #include "utils/zlocks.h"
 
 namespace dsn {
@@ -504,10 +505,9 @@ private:
 #endif
 
     // performance counters
-    perf_counter_wrapper _counter_replicas_count;
-    perf_counter_wrapper _counter_replicas_opening_count;
-    perf_counter_wrapper _counter_replicas_closing_count;
-    perf_counter_wrapper _counter_replicas_commit_qps;
+    METRIC_VAR_DECLARE_gauge_int64(total_replicas);
+    METRIC_VAR_DECLARE_gauge_int64(opening_replicas);
+    METRIC_VAR_DECLARE_gauge_int64(closing_replicas);
 
     perf_counter_wrapper _counter_replicas_learning_count;
     perf_counter_wrapper _counter_replicas_learning_max_duration_time_ms;
diff --git a/src/replica/replication_app_base.cpp b/src/replica/replication_app_base.cpp
index 5159fd757..33fa450e1 100644
--- a/src/replica/replication_app_base.cpp
+++ b/src/replica/replication_app_base.cpp
@@ -66,6 +66,11 @@
 #include "utils/threadpool_code.h"
 #include "utils/utils.h"
 
+METRIC_DEFINE_counter(replica,
+                      committed_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of committed requests");
+
 namespace dsn {
 class disk_file;
 
@@ -254,7 +259,8 @@ replication_app_base *replication_app_base::new_storage_instance(const std::stri
     return utils::factory_store<replication_app_base>::create(name.c_str(), PROVIDER_TYPE_MAIN, r);
 }
 
-replication_app_base::replication_app_base(replica *replica) : replica_base(replica)
+replication_app_base::replication_app_base(replica *replica)
+    : replica_base(replica), METRIC_VAR_INIT_replica(committed_requests)
 {
     _dir_data = utils::filesystem::path_combine(replica->dir(), "data");
     _dir_learn = utils::filesystem::path_combine(replica->dir(), "learn");
@@ -493,7 +499,7 @@ error_code replication_app_base::apply_mutation(const mutation *mu)
             "mutation {} committed on {}, batched_count = {}", mu->name(), str, batched_count);
     }
 
-    _replica->update_commit_qps(batched_count);
+    METRIC_VAR_INCREMENT_BY(committed_requests, batched_count);
 
     return ERR_OK;
 }
diff --git a/src/replica/replication_app_base.h b/src/replica/replication_app_base.h
index e48ef8a5d..e7ae8eff4 100644
--- a/src/replica/replication_app_base.h
+++ b/src/replica/replication_app_base.h
@@ -39,6 +39,7 @@
 #include "replica/replica_base.h"
 #include "replica_admin_types.h"
 #include "utils/error_code.h"
+#include "utils/metrics.h"
 #include "utils/ports.h"
 
 namespace dsn {
@@ -312,6 +313,9 @@ protected:
     replica_init_info _info;
 
     explicit replication_app_base(replication::replica *replica);
+
+private:
+    METRIC_VAR_DECLARE_counter(committed_requests);
 };
 
 } // namespace replication
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index 8b9d396a9..f9ab9c2dc 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -153,7 +153,8 @@ class error_code;
 //
 // Since a type tends to be a class template where there might be commas, use variadic arguments
 // instead of a single fixed argument to represent a type.
-#define METRIC_VAR_DECLARE(name, ...) __VA_ARGS__ _##name
+#define METRIC_VAR_NAME(name) _metric_##name
+#define METRIC_VAR_DECLARE(name, ...) __VA_ARGS__ METRIC_VAR_NAME(name)
 #define METRIC_VAR_DECLARE_gauge_int64(name) METRIC_VAR_DECLARE(name, dsn::gauge_ptr<int64_t>)
 #define METRIC_VAR_DECLARE_counter(name)                                                           \
     METRIC_VAR_DECLARE(name, dsn::counter_ptr<dsn::striped_long_adder, false>)
@@ -162,7 +163,7 @@ class error_code;
 
 // Initialize a metric variable in user class.
 #define METRIC_VAR_INIT(name, entity, ...)                                                         \
-    _##name(METRIC_##name.instantiate(entity##_metric_entity(), ##__VA_ARGS__))
+    METRIC_VAR_NAME(name)(METRIC_##name.instantiate(entity##_metric_entity(), ##__VA_ARGS__))
 #define METRIC_VAR_INIT_replica(name, ...) METRIC_VAR_INIT(name, replica, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_server(name, ...) METRIC_VAR_INIT(name, server, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_disk(name, ...) METRIC_VAR_INIT(name, disk, ##__VA_ARGS__)
@@ -175,15 +176,15 @@ class error_code;
     do {                                                                                           \
         const auto v = (x);                                                                        \
         if (v != 0) {                                                                              \
-            _##name->increment_by(v);                                                              \
+            METRIC_VAR_NAME(name)->increment_by(v);                                                \
         }                                                                                          \
     } while (0)
 
 // Perform increment() operations on gauges and counters.
-#define METRIC_VAR_INCREMENT(name) _##name->increment()
+#define METRIC_VAR_INCREMENT(name) METRIC_VAR_NAME(name)->increment()
 
 // Perform decrement() operations on gauges.
-#define METRIC_VAR_DECREMENT(name) _##name->decrement()
+#define METRIC_VAR_DECREMENT(name) METRIC_VAR_NAME(name)->decrement()
 
 // Perform set() operations on gauges and percentiles.
 //
@@ -191,38 +192,44 @@ class error_code;
 // * set(val): set a single value for a metric, such as gauge, percentile;
 // * set(n, val): set multiple repeated values (the number of duplicates is n) for a metric,
 // such as percentile.
-#define METRIC_VAR_SET(name, ...) _##name->set(__VA_ARGS__)
+#define METRIC_VAR_SET(name, ...) METRIC_VAR_NAME(name)->set(__VA_ARGS__)
 
 // Read the current measurement of gauges and counters.
-#define METRIC_VAR_VALUE(name) _##name->value()
+#define METRIC_VAR_VALUE(name) METRIC_VAR_NAME(name)->value()
 
 // Convenient macro that is used to compute latency automatically, which is dedicated to percentile.
 #define METRIC_VAR_AUTO_LATENCY(name, ...)                                                         \
-    dsn::auto_latency __##name##_auto_latency(_##name, ##__VA_ARGS__)
+    dsn::auto_latency __##name##_auto_latency(METRIC_VAR_NAME(name), ##__VA_ARGS__)
 
 #define METRIC_VAR_AUTO_LATENCY_DURATION_NS(name) __##name##_auto_latency.duration_ns()
 
 // Convenient macro that is used to increment/decrement gauge automatically in current scope.
 #define METRIC_VAR_AUTO_COUNT(name, ...)                                                           \
-    dsn::auto_count __##name##_auto_count(_##name, ##__VA_ARGS__)
+    dsn::auto_count __##name##_auto_count(METRIC_VAR_NAME(name), ##__VA_ARGS__)
+
+#define METRIC_FUNC_NAME_INCREMENT_BY(name) increment_##name##_by
 
 #define METRIC_DEFINE_INCREMENT_BY(name)                                                           \
-    void increment_##name##_by(int64_t x) { METRIC_VAR_INCREMENT_BY(name, x); }
+    void METRIC_FUNC_NAME_INCREMENT_BY(name)(int64_t x) { METRIC_VAR_INCREMENT_BY(name, x); }
 
 // To be adaptive to self-defined `increment_by` methods, arguments are declared as variadic.
-#define METRIC_INCREMENT_BY(obj, name, ...) (obj).increment_##name##_by(__VA_ARGS__)
+#define METRIC_INCREMENT_BY(obj, name, ...) (obj).METRIC_FUNC_NAME_INCREMENT_BY(name)(__VA_ARGS__)
+
+#define METRIC_FUNC_NAME_INCREMENT(name) increment_##name
 
 #define METRIC_DEFINE_INCREMENT(name)                                                              \
-    void increment_##name() { METRIC_VAR_INCREMENT(name); }
+    void METRIC_FUNC_NAME_INCREMENT(name)() { METRIC_VAR_INCREMENT(name); }
 
 // To be adaptive to self-defined `increment` methods, arguments are declared as variadic.
-#define METRIC_INCREMENT(obj, name, ...) (obj).increment_##name(__VA_ARGS__)
+#define METRIC_INCREMENT(obj, name, ...) (obj).METRIC_FUNC_NAME_INCREMENT(name)(__VA_ARGS__)
+
+#define METRIC_FUNC_NAME_SET(name) set_##name
 
 #define METRIC_DEFINE_SET(name, value_type)                                                        \
-    void set_##name(value_type value) { METRIC_VAR_SET(name, value); }
+    void METRIC_FUNC_NAME_SET(name)(value_type value) { METRIC_VAR_SET(name, value); }
 
 // To be adaptive to self-defined `set` methods, arguments are declared as variadic.
-#define METRIC_SET(obj, name, ...) (obj).set_##name(__VA_ARGS__)
+#define METRIC_SET(obj, name, ...) (obj).METRIC_FUNC_NAME_SET(name)(__VA_ARGS__)
 
 namespace dsn {
 class metric;                  // IWYU pragma: keep
@@ -646,6 +653,8 @@ enum class metric_unit : size_t
     kCapacityUnits,
     kPercent,
     kPartitions,
+    kReplicas,
+    kServers,
     kRequests,
     kSeeks,
     kPointLookups,
@@ -660,7 +669,6 @@ enum class metric_unit : size_t
     kOperations,
     kTasks,
     kDisconnections,
-    kServers,
     kInvalidUnit,
 };
 
diff --git a/src/utils/test/metrics_test.cpp b/src/utils/test/metrics_test.cpp
index 5197ec4b6..24b5d43e3 100644
--- a/src/utils/test/metrics_test.cpp
+++ b/src/utils/test/metrics_test.cpp
@@ -3083,12 +3083,12 @@ protected:
 
     void SetUp() override
     {
-        _test_replica_gauge_int64->set(0);
-        _test_replica_counter->reset();
-        _test_replica_percentile_int64_ns->reset_tail_for_test();
-        _test_replica_percentile_int64_us->reset_tail_for_test();
-        _test_replica_percentile_int64_ms->reset_tail_for_test();
-        _test_replica_percentile_int64_s->reset_tail_for_test();
+        METRIC_VAR_SET(test_replica_gauge_int64, 0);
+        METRIC_VAR_NAME(test_replica_counter)->reset();
+        METRIC_VAR_NAME(test_replica_percentile_int64_ns)->reset_tail_for_test();
+        METRIC_VAR_NAME(test_replica_percentile_int64_us)->reset_tail_for_test();
+        METRIC_VAR_NAME(test_replica_percentile_int64_ms)->reset_tail_for_test();
+        METRIC_VAR_NAME(test_replica_percentile_int64_s)->reset_tail_for_test();
     }
 
     const metric_entity_ptr &my_replica_metric_entity() const { return _my_replica_metric_entity; }
@@ -3120,7 +3120,7 @@ MetricVarTest::MetricVarTest()
 {
 }
 
-#define METRIC_VAR_SAMPLES(name) _##name->samples_for_test()
+#define METRIC_VAR_SAMPLES(name) METRIC_VAR_NAME(name)->samples_for_test()
 
 void MetricVarTest::test_set_percentile(const std::vector<int64_t> &expected_samples)
 {


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


[incubator-pegasus] 03/23: feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 1) (#1374)

Posted by wa...@apache.org.
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 624ed5c1922364107377bdf633803ee75f2ab767
Author: Dan Wang <wa...@apache.org>
AuthorDate: Tue Mar 7 14:32:56 2023 +0800

    feat(new_metrics): migrate replica-level metrics for pegasus_server_impl (part 1) (#1374)
    
    This PR is to migrate replica-level metrics of `pegasus_server_impl` to new framework
    (https://github.com/apache/incubator-pegasus/issues/1333). Since there are many replica-level
    metrics in `pegasus_server_impl`, this PR is part 1 for this migration and other metrics (all
    of which are gauges) will be migrated in the later PRs.
---
 src/server/pegasus_server_impl.cpp           | 250 ++++++++++++++-------------
 src/server/pegasus_server_impl.h             |  32 ++--
 src/server/pegasus_server_impl_init.cpp      | 126 +++++++-------
 src/server/pegasus_write_service_impl.h      |   5 +-
 src/server/rocksdb_wrapper.cpp               |   6 +-
 src/server/rocksdb_wrapper.h                 |   2 +-
 src/server/test/pegasus_server_impl_test.cpp |   4 +-
 src/utils/metrics.h                          |  16 +-
 src/utils/test/metrics_test.cpp              |   2 -
 src/utils/time_utils.h                       |   2 +-
 10 files changed, 242 insertions(+), 203 deletions(-)

diff --git a/src/server/pegasus_server_impl.cpp b/src/server/pegasus_server_impl.cpp
index e6e9d48ed..ab6c4dc1c 100644
--- a/src/server/pegasus_server_impl.cpp
+++ b/src/server/pegasus_server_impl.cpp
@@ -322,13 +322,50 @@ int pegasus_server_impl::on_batched_write_requests(int64_t decree,
     return _server_write->on_batched_write_requests(requests, count, decree, timestamp);
 }
 
+// Since LOG_ERROR_PREFIX depends on log_prefix(), this method could not be declared as static or
+// with anonymous namespace.
+void pegasus_server_impl::log_expired_data(const char *op,
+                                           const dsn::rpc_address &addr,
+                                           const dsn::blob &hash_key,
+                                           const dsn::blob &sort_key) const
+{
+    LOG_ERROR_PREFIX("rocksdb data expired for {} from {}: hash_key = \"{}\", sort_key = \"{}\"",
+                     op,
+                     addr,
+                     pegasus::utils::c_escape_string(hash_key),
+                     pegasus::utils::c_escape_string(sort_key));
+}
+
+void pegasus_server_impl::log_expired_data(const char *op,
+                                           const dsn::rpc_address &addr,
+                                           const dsn::blob &key) const
+{
+    dsn::blob hash_key, sort_key;
+    pegasus_restore_key(key, hash_key, sort_key);
+    log_expired_data(op, addr, hash_key, sort_key);
+}
+
+void pegasus_server_impl::log_expired_data(const char *op,
+                                           const dsn::rpc_address &addr,
+                                           const rocksdb::Slice &key) const
+{
+    dsn::blob raw_key(key.data(), 0, key.size());
+    log_expired_data(op, addr, raw_key);
+}
+
+#define LOG_EXPIRED_DATA_IF_VERBOSE(...)                                                           \
+    do {                                                                                           \
+        if (dsn_unlikely(FLAGS_rocksdb_verbose_log)) {                                             \
+            log_expired_data(__FUNCTION__, rpc.remote_address(), ##__VA_ARGS__);                   \
+        }                                                                                          \
+    } while (0)
+
 void pegasus_server_impl::on_get(get_rpc rpc)
 {
-    CHECK(_is_open, "");
-    _pfc_get_qps->increment();
-    uint64_t start_time = dsn_now_ns();
+    CHECK_TRUE(_is_open);
+
+    METRIC_VAR_INCREMENT(get_requests);
 
-    const auto &key = rpc.request();
     auto &resp = rpc.response();
     resp.app_id = _gpid.get_app_id();
     resp.partition_index = _gpid.get_partition_index();
@@ -345,21 +382,20 @@ void pegasus_server_impl::on_get(get_rpc rpc)
         return;
     }
 
+    METRIC_VAR_AUTO_LATENCY(get_latency_ns);
+
+    const auto &key = rpc.request();
     rocksdb::Slice skey(key.data(), key.length());
     std::string value;
     rocksdb::Status status = _db->Get(_data_cf_rd_opts, _data_cf, skey, &value);
 
     if (status.ok()) {
         if (check_if_record_expired(utils::epoch_now(), value)) {
-            _pfc_recent_expire_count->increment();
-            if (FLAGS_rocksdb_verbose_log) {
-                LOG_ERROR_PREFIX("rocksdb data expired for get from {}", rpc.remote_address());
-            }
+            METRIC_VAR_INCREMENT(read_expired_values);
+            LOG_EXPIRED_DATA_IF_VERBOSE(key);
             status = rocksdb::Status::NotFound();
         }
-    }
-
-    if (!status.ok()) {
+    } else {
         if (FLAGS_rocksdb_verbose_log) {
             ::dsn::blob hash_key, sort_key;
             pegasus_restore_key(key, hash_key, sort_key);
@@ -382,7 +418,7 @@ void pegasus_server_impl::on_get(get_rpc rpc)
     usleep(10 * 1000);
 #endif
 
-    uint64_t time_used = dsn_now_ns() - start_time;
+    auto time_used = METRIC_VAR_AUTO_LATENCY_DURATION_NS(get_latency_ns);
     if (is_get_abnormal(time_used, value.size())) {
         ::dsn::blob hash_key, sort_key;
         pegasus_restore_key(key, hash_key, sort_key);
@@ -395,7 +431,7 @@ void pegasus_server_impl::on_get(get_rpc rpc)
                            status.ToString(),
                            value.size(),
                            time_used);
-        _pfc_recent_abnormal_count->increment();
+        METRIC_VAR_INCREMENT(abnormal_read_requests);
     }
 
     resp.error = status.code();
@@ -404,17 +440,14 @@ void pegasus_server_impl::on_get(get_rpc rpc)
     }
 
     _cu_calculator->add_get_cu(rpc.dsn_request(), resp.error, key, resp.value);
-    _pfc_get_latency->set(dsn_now_ns() - start_time);
 }
 
 void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
 {
-    CHECK(_is_open, "");
-    _pfc_multi_get_qps->increment();
-    uint64_t start_time = dsn_now_ns();
+    CHECK_TRUE(_is_open);
+
+    METRIC_VAR_INCREMENT(multi_get_requests);
 
-    const auto &request = rpc.request();
-    dsn::message_ex *req = rpc.dsn_request();
     auto &resp = rpc.response();
     resp.app_id = _gpid.get_app_id();
     resp.partition_index = _gpid.get_partition_index();
@@ -426,6 +459,10 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
         return;
     }
 
+    METRIC_VAR_AUTO_LATENCY(multi_get_latency_ns);
+
+    const auto &request = rpc.request();
+    dsn::message_ex *req = rpc.dsn_request();
     if (!is_filter_type_supported(request.sort_key_filter_type)) {
         LOG_ERROR_PREFIX(
             "invalid argument for multi_get from {}: sort key filter type {} not supported",
@@ -433,7 +470,6 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
             request.sort_key_filter_type);
         resp.error = rocksdb::Status::kInvalidArgument;
         _cu_calculator->add_multi_get_cu(req, resp.error, request.hash_key, resp.kvs);
-        _pfc_multi_get_latency->set(dsn_now_ns() - start_time);
         return;
     }
 
@@ -520,8 +556,6 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
             }
             resp.error = rocksdb::Status::kOk;
             _cu_calculator->add_multi_get_cu(req, resp.error, request.hash_key, resp.kvs);
-            _pfc_multi_get_latency->set(dsn_now_ns() - start_time);
-
             return;
         }
 
@@ -724,7 +758,6 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
         for (int i = 0; i < keys.size(); i++) {
             rocksdb::Status &status = statuses[i];
             std::string &value = values[i];
-            // print log
             if (!status.ok()) {
                 if (FLAGS_rocksdb_verbose_log) {
                     LOG_ERROR_PREFIX("rocksdb get failed for multi_get from {}: hash_key = \"{}\", "
@@ -738,41 +771,38 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
                                      rpc.remote_address(),
                                      status.ToString());
                 }
-            }
-            // check ttl
-            if (status.ok()) {
-                uint32_t expire_ts = pegasus_extract_expire_ts(_pegasus_data_version, value);
-                if (expire_ts > 0 && expire_ts <= epoch_now) {
-                    expire_count++;
-                    if (FLAGS_rocksdb_verbose_log) {
-                        LOG_ERROR_PREFIX("rocksdb data expired for multi_get from {}",
-                                         rpc.remote_address());
-                    }
-                    status = rocksdb::Status::NotFound();
-                }
-            }
-            // extract value
-            if (status.ok()) {
-                // check if exceed limit
-                if (count >= max_kv_count || size >= max_kv_size) {
-                    exceed_limit = true;
-                    break;
-                }
-                ::dsn::apps::key_value kv;
-                kv.key = request.sort_keys[i];
-                if (!request.no_value) {
-                    pegasus_extract_user_data(_pegasus_data_version, std::move(value), kv.value);
+
+                if (status.IsNotFound()) {
+                    continue;
                 }
-                count++;
-                size += kv.key.length() + kv.value.length();
-                resp.kvs.emplace_back(std::move(kv));
-            }
-            // if error occurred
-            if (!status.ok() && !status.IsNotFound()) {
+
                 error_occurred = true;
                 final_status = status;
                 break;
             }
+
+            // check ttl
+            if (check_if_record_expired(epoch_now, value)) {
+                expire_count++;
+                LOG_EXPIRED_DATA_IF_VERBOSE(request.hash_key, request.sort_keys[i]);
+                status = rocksdb::Status::NotFound();
+                continue;
+            }
+
+            // check if exceed limit
+            if (dsn_unlikely(count >= max_kv_count || size >= max_kv_size)) {
+                exceed_limit = true;
+                break;
+            }
+
+            ::dsn::apps::key_value kv;
+            kv.key = request.sort_keys[i];
+            if (!request.no_value) {
+                pegasus_extract_user_data(_pegasus_data_version, std::move(value), kv.value);
+            }
+            count++;
+            size += kv.key.length() + kv.value.length();
+            resp.kvs.emplace_back(std::move(kv));
         }
 
         if (error_occurred) {
@@ -790,7 +820,7 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
     usleep(10 * 1000);
 #endif
 
-    uint64_t time_used = dsn_now_ns() - start_time;
+    auto time_used = METRIC_VAR_AUTO_LATENCY_DURATION_NS(multi_get_latency_ns);
     if (is_multi_get_abnormal(time_used, size, iteration_count)) {
         LOG_WARNING_PREFIX(
             "rocksdb abnormal multi_get from {}: hash_key = {}, "
@@ -816,25 +846,20 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
             expire_count,
             filter_count,
             time_used);
-        _pfc_recent_abnormal_count->increment();
+        METRIC_VAR_INCREMENT(abnormal_read_requests);
     }
 
-    if (expire_count > 0) {
-        _pfc_recent_expire_count->add(expire_count);
-    }
-    if (filter_count > 0) {
-        _pfc_recent_filter_count->add(filter_count);
-    }
+    METRIC_VAR_INCREMENT_BY(read_expired_values, expire_count);
+    METRIC_VAR_INCREMENT_BY(read_filtered_values, filter_count);
 
     _cu_calculator->add_multi_get_cu(req, resp.error, request.hash_key, resp.kvs);
-    _pfc_multi_get_latency->set(dsn_now_ns() - start_time);
 }
 
 void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
 {
-    CHECK(_is_open, "");
-    _pfc_batch_get_qps->increment();
-    int64_t start_time = dsn_now_ns();
+    CHECK_TRUE(_is_open);
+
+    METRIC_VAR_INCREMENT(batch_get_requests);
 
     auto &response = rpc.response();
     response.app_id = _gpid.get_app_id();
@@ -847,13 +872,14 @@ void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
         return;
     }
 
+    METRIC_VAR_AUTO_LATENCY(batch_get_latency_ns);
+
     const auto &request = rpc.request();
     if (request.keys.empty()) {
         response.error = rocksdb::Status::kInvalidArgument;
         LOG_ERROR_PREFIX("Invalid argument for batch_get from {}: 'keys' field in request is empty",
                          rpc.remote_address().to_string());
         _cu_calculator->add_batch_get_cu(rpc.dsn_request(), response.error, response.data);
-        _pfc_batch_get_latency->set(dsn_now_ns() - start_time);
         return;
     }
 
@@ -872,6 +898,8 @@ void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
     bool error_occurred = false;
     int64_t total_data_size = 0;
     uint32_t epoch_now = pegasus::utils::epoch_now();
+    uint64_t expire_count = 0;
+
     std::vector<std::string> values;
     std::vector<rocksdb::Status> statuses = _db->MultiGet(_data_cf_rd_opts, keys, &values);
     response.data.reserve(request.keys.size());
@@ -887,13 +915,8 @@ void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
 
         if (dsn_likely(status.ok())) {
             if (check_if_record_expired(epoch_now, value)) {
-                if (FLAGS_rocksdb_verbose_log) {
-                    LOG_ERROR_PREFIX(
-                        "rocksdb data expired for batch_get from {}, hash_key = {}, sort_key = {}",
-                        rpc.remote_address().to_string(),
-                        pegasus::utils::c_escape_string(hash_key),
-                        pegasus::utils::c_escape_string(sort_key));
-                }
+                ++expire_count;
+                LOG_EXPIRED_DATA_IF_VERBOSE(hash_key, sort_key);
                 continue;
             }
 
@@ -931,7 +954,7 @@ void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
         response.error = rocksdb::Status::kOk;
     }
 
-    int64_t time_used = dsn_now_ns() - start_time;
+    auto time_used = METRIC_VAR_AUTO_LATENCY_DURATION_NS(batch_get_latency_ns);
     if (is_batch_get_abnormal(time_used, total_data_size, request.keys.size())) {
         LOG_WARNING_PREFIX(
             "rocksdb abnormal batch_get from {}: total data size = {}, row count = {}, "
@@ -940,33 +963,36 @@ void pegasus_server_impl::on_batch_get(batch_get_rpc rpc)
             total_data_size,
             request.keys.size(),
             time_used / 1000);
-        _pfc_recent_abnormal_count->increment();
+        METRIC_VAR_INCREMENT(abnormal_read_requests);
     }
 
+    METRIC_VAR_INCREMENT_BY(read_expired_values, expire_count);
+
     _cu_calculator->add_batch_get_cu(rpc.dsn_request(), response.error, response.data);
-    _pfc_batch_get_latency->set(time_used);
 }
 
 void pegasus_server_impl::on_sortkey_count(sortkey_count_rpc rpc)
 {
-    CHECK(_is_open, "");
+    CHECK_TRUE(_is_open);
 
-    _pfc_scan_qps->increment();
-    uint64_t start_time = dsn_now_ns();
+    METRIC_VAR_INCREMENT(scan_requests);
 
-    const auto &hash_key = rpc.request();
     auto &resp = rpc.response();
     resp.app_id = _gpid.get_app_id();
     resp.partition_index = _gpid.get_partition_index();
     resp.server = _primary_address;
+
     if (!_read_size_throttling_controller->available()) {
         rpc.error() = dsn::ERR_BUSY;
         _counter_recent_read_throttling_reject_count->increment();
         return;
     }
 
+    METRIC_VAR_AUTO_LATENCY(scan_latency_ns);
+
     // scan
     ::dsn::blob start_key, stop_key;
+    const auto &hash_key = rpc.request();
     pegasus_generate_key(start_key, hash_key, ::dsn::blob());
     pegasus_generate_next_blob(stop_key, hash_key);
     rocksdb::Slice start(start_key.data(), start_key.length());
@@ -988,18 +1014,14 @@ void pegasus_server_impl::on_sortkey_count(sortkey_count_rpc rpc)
 
         if (check_if_record_expired(epoch_now, it->value())) {
             expire_count++;
-            if (FLAGS_rocksdb_verbose_log) {
-                LOG_ERROR_PREFIX("rocksdb data expired for sortkey_count from {}",
-                                 rpc.remote_address());
-            }
+            LOG_EXPIRED_DATA_IF_VERBOSE(it->key());
         } else {
             resp.count++;
         }
         it->Next();
     }
-    if (expire_count > 0) {
-        _pfc_recent_expire_count->add(expire_count);
-    }
+
+    METRIC_VAR_INCREMENT_BY(read_expired_values, expire_count);
 
     resp.error = it->status().code();
     if (!it->status().ok()) {
@@ -1025,7 +1047,6 @@ void pegasus_server_impl::on_sortkey_count(sortkey_count_rpc rpc)
     }
 
     _cu_calculator->add_sortkey_count_cu(rpc.dsn_request(), resp.error, hash_key);
-    _pfc_scan_latency->set(dsn_now_ns() - start_time);
 }
 
 void pegasus_server_impl::on_ttl(ttl_rpc rpc)
@@ -1053,7 +1074,7 @@ void pegasus_server_impl::on_ttl(ttl_rpc rpc)
     if (status.ok()) {
         expire_ts = pegasus_extract_expire_ts(_pegasus_data_version, value);
         if (check_if_ts_expired(now_ts, expire_ts)) {
-            _pfc_recent_expire_count->increment();
+            METRIC_VAR_INCREMENT(read_expired_values);
             if (FLAGS_rocksdb_verbose_log) {
                 LOG_ERROR_PREFIX("rocksdb data expired for ttl from {}", rpc.remote_address());
             }
@@ -1093,12 +1114,10 @@ void pegasus_server_impl::on_ttl(ttl_rpc rpc)
 
 void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
 {
-    CHECK(_is_open, "");
-    _pfc_scan_qps->increment();
-    uint64_t start_time = dsn_now_ns();
+    CHECK_TRUE(_is_open);
+
+    METRIC_VAR_INCREMENT(scan_requests);
 
-    const auto &request = rpc.request();
-    dsn::message_ex *req = rpc.dsn_request();
     auto &resp = rpc.response();
     resp.app_id = _gpid.get_app_id();
     resp.partition_index = _gpid.get_partition_index();
@@ -1110,6 +1129,10 @@ void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
         return;
     }
 
+    METRIC_VAR_AUTO_LATENCY(scan_latency_ns);
+
+    const auto &request = rpc.request();
+    dsn::message_ex *req = rpc.dsn_request();
     if (!is_filter_type_supported(request.hash_key_filter_type)) {
         LOG_ERROR_PREFIX(
             "invalid argument for get_scanner from {}: hash key filter type {} not supported",
@@ -1117,10 +1140,9 @@ void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
             request.hash_key_filter_type);
         resp.error = rocksdb::Status::kInvalidArgument;
         _cu_calculator->add_scan_cu(req, resp.error, resp.kvs);
-        _pfc_scan_latency->set(dsn_now_ns() - start_time);
-
         return;
     }
+
     if (!is_filter_type_supported(request.sort_key_filter_type)) {
         LOG_ERROR_PREFIX(
             "invalid argument for get_scanner from {}: sort key filter type {} not supported",
@@ -1128,8 +1150,6 @@ void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
             request.sort_key_filter_type);
         resp.error = rocksdb::Status::kInvalidArgument;
         _cu_calculator->add_scan_cu(req, resp.error, resp.kvs);
-        _pfc_scan_latency->set(dsn_now_ns() - start_time);
-
         return;
     }
 
@@ -1185,8 +1205,6 @@ void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
         }
         resp.error = rocksdb::Status::kOk;
         _cu_calculator->add_scan_cu(req, resp.error, resp.kvs);
-        _pfc_scan_latency->set(dsn_now_ns() - start_time);
-
         return;
     }
 
@@ -1339,24 +1357,18 @@ void pegasus_server_impl::on_get_scanner(get_scanner_rpc rpc)
         resp.context_id = pegasus::SCAN_CONTEXT_ID_COMPLETED;
     }
 
-    if (expire_count > 0) {
-        _pfc_recent_expire_count->add(expire_count);
-    }
-    if (filter_count > 0) {
-        _pfc_recent_filter_count->add(filter_count);
-    }
+    METRIC_VAR_INCREMENT_BY(read_expired_values, expire_count);
+    METRIC_VAR_INCREMENT_BY(read_filtered_values, filter_count);
 
     _cu_calculator->add_scan_cu(req, resp.error, resp.kvs);
-    _pfc_scan_latency->set(dsn_now_ns() - start_time);
 }
 
 void pegasus_server_impl::on_scan(scan_rpc rpc)
 {
-    CHECK(_is_open, "");
-    _pfc_scan_qps->increment();
-    uint64_t start_time = dsn_now_ns();
-    const auto &request = rpc.request();
-    dsn::message_ex *req = rpc.dsn_request();
+    CHECK_TRUE(_is_open);
+
+    METRIC_VAR_INCREMENT(scan_requests);
+
     auto &resp = rpc.response();
     resp.app_id = _gpid.get_app_id();
     resp.partition_index = _gpid.get_partition_index();
@@ -1368,6 +1380,10 @@ void pegasus_server_impl::on_scan(scan_rpc rpc)
         return;
     }
 
+    METRIC_VAR_AUTO_LATENCY(scan_latency_ns);
+
+    const auto &request = rpc.request();
+    dsn::message_ex *req = rpc.dsn_request();
     std::unique_ptr<pegasus_scan_context> context = _context_cache.fetch(request.context_id);
     if (context) {
         rocksdb::Iterator *it = context->iterator.get();
@@ -1490,18 +1506,14 @@ void pegasus_server_impl::on_scan(scan_rpc rpc)
             resp.context_id = pegasus::SCAN_CONTEXT_ID_COMPLETED;
         }
 
-        if (expire_count > 0) {
-            _pfc_recent_expire_count->add(expire_count);
-        }
-        if (filter_count > 0) {
-            _pfc_recent_filter_count->add(filter_count);
-        }
+        METRIC_VAR_INCREMENT_BY(read_expired_values, expire_count);
+        METRIC_VAR_INCREMENT_BY(read_filtered_values, filter_count);
+
     } else {
         resp.error = rocksdb::Status::Code::kNotFound;
     }
 
     _cu_calculator->add_scan_cu(req, resp.error, resp.kvs);
-    _pfc_scan_latency->set(dsn_now_ns() - start_time);
 }
 
 void pegasus_server_impl::on_clear_scanner(const int64_t &args) { _context_cache.fetch(args); }
diff --git a/src/server/pegasus_server_impl.h b/src/server/pegasus_server_impl.h
index d156acdce..7203d0ce2 100644
--- a/src/server/pegasus_server_impl.h
+++ b/src/server/pegasus_server_impl.h
@@ -446,6 +446,15 @@ private:
 
     dsn::replication::manual_compaction_status::type query_compact_status() const override;
 
+    // Log expired keys for verbose mode.
+    void log_expired_data(const char *op,
+                          const dsn::rpc_address &addr,
+                          const dsn::blob &hash_key,
+                          const dsn::blob &sort_key) const;
+    void log_expired_data(const char *op, const dsn::rpc_address &addr, const dsn::blob &key) const;
+    void
+    log_expired_data(const char *op, const dsn::rpc_address &addr, const rocksdb::Slice &key) const;
+
 private:
     static const std::chrono::seconds kServerStatUpdateTimeSec;
     static const std::string COMPRESSION_HEADER;
@@ -517,20 +526,19 @@ private:
 
     std::shared_ptr<throttling_controller> _read_size_throttling_controller;
 
-    // perf counters
-    ::dsn::perf_counter_wrapper _pfc_get_qps;
-    ::dsn::perf_counter_wrapper _pfc_multi_get_qps;
-    ::dsn::perf_counter_wrapper _pfc_batch_get_qps;
-    ::dsn::perf_counter_wrapper _pfc_scan_qps;
+    METRIC_VAR_DECLARE_counter(get_requests);
+    METRIC_VAR_DECLARE_counter(multi_get_requests);
+    METRIC_VAR_DECLARE_counter(batch_get_requests);
+    METRIC_VAR_DECLARE_counter(scan_requests);
 
-    ::dsn::perf_counter_wrapper _pfc_get_latency;
-    ::dsn::perf_counter_wrapper _pfc_multi_get_latency;
-    ::dsn::perf_counter_wrapper _pfc_batch_get_latency;
-    ::dsn::perf_counter_wrapper _pfc_scan_latency;
+    METRIC_VAR_DECLARE_percentile_int64(get_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(multi_get_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(batch_get_latency_ns);
+    METRIC_VAR_DECLARE_percentile_int64(scan_latency_ns);
 
-    ::dsn::perf_counter_wrapper _pfc_recent_expire_count;
-    ::dsn::perf_counter_wrapper _pfc_recent_filter_count;
-    ::dsn::perf_counter_wrapper _pfc_recent_abnormal_count;
+    METRIC_VAR_DECLARE_counter(read_expired_values);
+    METRIC_VAR_DECLARE_counter(read_filtered_values);
+    METRIC_VAR_DECLARE_counter(abnormal_read_requests);
 
     // rocksdb internal statistics
     // server level
diff --git a/src/server/pegasus_server_impl_init.cpp b/src/server/pegasus_server_impl_init.cpp
index 896f8ab94..27aebdfab 100644
--- a/src/server/pegasus_server_impl_init.cpp
+++ b/src/server/pegasus_server_impl_init.cpp
@@ -62,6 +62,61 @@ class replica;
 } // namespace replication
 } // namespace dsn
 
+METRIC_DEFINE_counter(replica,
+                      get_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of GET requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      multi_get_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of MULTI_GET requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      batch_get_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of BATCH_GET requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      scan_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of SCAN requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               get_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of GET requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               multi_get_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of MULTI_GET requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               batch_get_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of BATCH_GET requests for each replica");
+
+METRIC_DEFINE_percentile_int64(replica,
+                               scan_latency_ns,
+                               dsn::metric_unit::kNanoSeconds,
+                               "The latency of SCAN requests for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      read_expired_values,
+                      dsn::metric_unit::kValues,
+                      "The number of expired values read for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      read_filtered_values,
+                      dsn::metric_unit::kValues,
+                      "The number of filtered values read for each replica");
+
+METRIC_DEFINE_counter(replica,
+                      abnormal_read_requests,
+                      dsn::metric_unit::kRequests,
+                      "The number of abnormal read requests for each replica");
+
 namespace pegasus {
 namespace server {
 
@@ -403,7 +458,18 @@ pegasus_server_impl::pegasus_server_impl(dsn::replication::replica *r)
       _last_durable_decree(0),
       _is_checkpointing(false),
       _manual_compact_svc(this),
-      _partition_version(0)
+      _partition_version(0),
+      METRIC_VAR_INIT_replica(get_requests),
+      METRIC_VAR_INIT_replica(multi_get_requests),
+      METRIC_VAR_INIT_replica(batch_get_requests),
+      METRIC_VAR_INIT_replica(scan_requests),
+      METRIC_VAR_INIT_replica(get_latency_ns),
+      METRIC_VAR_INIT_replica(multi_get_latency_ns),
+      METRIC_VAR_INIT_replica(batch_get_latency_ns),
+      METRIC_VAR_INIT_replica(scan_latency_ns),
+      METRIC_VAR_INIT_replica(read_expired_values),
+      METRIC_VAR_INIT_replica(read_filtered_values),
+      METRIC_VAR_INIT_replica(abnormal_read_requests)
 {
     _primary_address = dsn::rpc_address(dsn_primary_address()).to_string();
     _gpid = get_gpid();
@@ -610,64 +676,6 @@ pegasus_server_impl::pegasus_server_impl(dsn::replication::replica *r)
     char name[256];
 
     // register the perf counters
-    snprintf(name, 255, "get_qps@%s", str_gpid.c_str());
-    _pfc_get_qps.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the qps of GET request");
-
-    snprintf(name, 255, "multi_get_qps@%s", str_gpid.c_str());
-    _pfc_multi_get_qps.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the qps of MULTI_GET request");
-
-    snprintf(name, 255, "batch_get_qps@%s", str_gpid.c_str());
-    _pfc_batch_get_qps.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the qps of BATCH_GET request");
-
-    snprintf(name, 255, "scan_qps@%s", str_gpid.c_str());
-    _pfc_scan_qps.init_app_counter(
-        "app.pegasus", name, COUNTER_TYPE_RATE, "statistic the qps of SCAN request");
-
-    snprintf(name, 255, "get_latency@%s", str_gpid.c_str());
-    _pfc_get_latency.init_app_counter("app.pegasus",
-                                      name,
-                                      COUNTER_TYPE_NUMBER_PERCENTILES,
-                                      "statistic the latency of GET request");
-
-    snprintf(name, 255, "multi_get_latency@%s", str_gpid.c_str());
-    _pfc_multi_get_latency.init_app_counter("app.pegasus",
-                                            name,
-                                            COUNTER_TYPE_NUMBER_PERCENTILES,
-                                            "statistic the latency of MULTI_GET request");
-
-    snprintf(name, 255, "batch_get_latency@%s", str_gpid.c_str());
-    _pfc_batch_get_latency.init_app_counter("app.pegasus",
-                                            name,
-                                            COUNTER_TYPE_NUMBER_PERCENTILES,
-                                            "statistic the latency of BATCH_GET request");
-
-    snprintf(name, 255, "scan_latency@%s", str_gpid.c_str());
-    _pfc_scan_latency.init_app_counter("app.pegasus",
-                                       name,
-                                       COUNTER_TYPE_NUMBER_PERCENTILES,
-                                       "statistic the latency of SCAN request");
-
-    snprintf(name, 255, "recent.expire.count@%s", str_gpid.c_str());
-    _pfc_recent_expire_count.init_app_counter("app.pegasus",
-                                              name,
-                                              COUNTER_TYPE_VOLATILE_NUMBER,
-                                              "statistic the recent expired value read count");
-
-    snprintf(name, 255, "recent.filter.count@%s", str_gpid.c_str());
-    _pfc_recent_filter_count.init_app_counter("app.pegasus",
-                                              name,
-                                              COUNTER_TYPE_VOLATILE_NUMBER,
-                                              "statistic the recent filtered value read count");
-
-    snprintf(name, 255, "recent.abnormal.count@%s", str_gpid.c_str());
-    _pfc_recent_abnormal_count.init_app_counter("app.pegasus",
-                                                name,
-                                                COUNTER_TYPE_VOLATILE_NUMBER,
-                                                "statistic the recent abnormal read count");
-
     snprintf(name, 255, "disk.storage.sst.count@%s", str_gpid.c_str());
     _pfc_rdb_sst_count.init_app_counter(
         "app.pegasus", name, COUNTER_TYPE_NUMBER, "statistic the count of sstable files");
diff --git a/src/server/pegasus_write_service_impl.h b/src/server/pegasus_write_service_impl.h
index b75b379a3..344c89560 100644
--- a/src/server/pegasus_write_service_impl.h
+++ b/src/server/pegasus_write_service_impl.h
@@ -89,8 +89,7 @@ public:
     explicit impl(pegasus_server_impl *server)
         : replica_base(server),
           _primary_address(server->_primary_address),
-          _pegasus_data_version(server->_pegasus_data_version),
-          _pfc_recent_expire_count(server->_pfc_recent_expire_count)
+          _pegasus_data_version(server->_pegasus_data_version)
     {
         _rocksdb_wrapper = std::make_unique<rocksdb_wrapper>(server);
     }
@@ -689,8 +688,6 @@ private:
     const std::string _primary_address;
     const uint32_t _pegasus_data_version;
 
-    ::dsn::perf_counter_wrapper &_pfc_recent_expire_count;
-
     std::unique_ptr<rocksdb_wrapper> _rocksdb_wrapper;
 
     // for setting update_response.error after committed.
diff --git a/src/server/rocksdb_wrapper.cpp b/src/server/rocksdb_wrapper.cpp
index 64d37d9be..08fe8aeca 100644
--- a/src/server/rocksdb_wrapper.cpp
+++ b/src/server/rocksdb_wrapper.cpp
@@ -39,6 +39,8 @@
 #include "utils/fmt_logging.h"
 #include "utils/ports.h"
 
+METRIC_DECLARE_counter(read_expired_values);
+
 namespace pegasus {
 namespace server {
 
@@ -54,7 +56,7 @@ rocksdb_wrapper::rocksdb_wrapper(pegasus_server_impl *server)
       _rd_opts(server->_data_cf_rd_opts),
       _meta_cf(server->_meta_cf),
       _pegasus_data_version(server->_pegasus_data_version),
-      _pfc_recent_expire_count(server->_pfc_recent_expire_count),
+      METRIC_VAR_INIT_replica(read_expired_values),
       _default_ttl(0)
 {
     _write_batch = std::make_unique<rocksdb::WriteBatch>();
@@ -76,7 +78,7 @@ int rocksdb_wrapper::get(dsn::string_view raw_key, /*out*/ db_get_context *ctx)
         ctx->expire_ts = pegasus_extract_expire_ts(_pegasus_data_version, ctx->raw_value);
         if (check_if_ts_expired(utils::epoch_now(), ctx->expire_ts)) {
             ctx->expired = true;
-            _pfc_recent_expire_count->increment();
+            METRIC_VAR_INCREMENT(read_expired_values);
         }
         return rocksdb::Status::kOk;
     } else if (s.IsNotFound()) {
diff --git a/src/server/rocksdb_wrapper.h b/src/server/rocksdb_wrapper.h
index e3c713512..fef30a629 100644
--- a/src/server/rocksdb_wrapper.h
+++ b/src/server/rocksdb_wrapper.h
@@ -87,7 +87,7 @@ private:
     rocksdb::ColumnFamilyHandle *_meta_cf;
 
     const uint32_t _pegasus_data_version;
-    dsn::perf_counter_wrapper &_pfc_recent_expire_count;
+    METRIC_VAR_DECLARE_counter(read_expired_values);
     volatile uint32_t _default_ttl;
 
     friend class rocksdb_wrapper_test;
diff --git a/src/server/test/pegasus_server_impl_test.cpp b/src/server/test/pegasus_server_impl_test.cpp
index 1363d8054..075193fef 100644
--- a/src/server/test/pegasus_server_impl_test.cpp
+++ b/src/server/test/pegasus_server_impl_test.cpp
@@ -77,7 +77,7 @@ public:
             _server->update_app_envs(envs);
 
             // do on_get/on_multi_get operation,
-            long before_count = _server->_pfc_recent_abnormal_count->get_integer_value();
+            auto before_count = _server->METRIC_VAR_VALUE(abnormal_read_requests);
             if (!test.is_multi_get) {
                 get_rpc rpc(std::make_unique<dsn::blob>(test_key), dsn::apps::RPC_RRDB_RRDB_GET);
                 _server->on_get(rpc);
@@ -90,7 +90,7 @@ public:
                                   dsn::apps::RPC_RRDB_RRDB_MULTI_GET);
                 _server->on_multi_get(rpc);
             }
-            long after_count = _server->_pfc_recent_abnormal_count->get_integer_value();
+            auto after_count = _server->METRIC_VAR_VALUE(abnormal_read_requests);
 
             ASSERT_EQ(before_count + test.expect_perf_counter_incr, after_count);
         }
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index 2d6da6c0f..da1d056d8 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -165,7 +165,13 @@ class error_code;
 #define METRIC_VAR_INIT_replica(name) METRIC_VAR_INIT(name, replica)
 
 // Perform increment-related operations on metrics including gauge and counter.
-#define METRIC_VAR_INCREMENT_BY(name, x) _##name->increment_by(x)
+#define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
+    do {                                                                                           \
+        if (x != 0) {                                                                              \
+            _##name->increment_by(x);                                                              \
+        }                                                                                          \
+    } while (0)
+
 #define METRIC_VAR_INCREMENT(name) _##name->increment()
 
 // Perform set() operations on metrics including gauge and percentile.
@@ -176,10 +182,15 @@ class error_code;
 // such as percentile.
 #define METRIC_VAR_SET(name, ...) _##name->set(__VA_ARGS__)
 
+// Read the current measurement of the metric.
+#define METRIC_VAR_VALUE(name) _##name->value()
+
 // Convenient macro that is used to compute latency automatically, which is dedicated to percentile.
 #define METRIC_VAR_AUTO_LATENCY(name, ...)                                                         \
     dsn::auto_latency __##name##_auto_latency(_##name, ##__VA_ARGS__)
 
+#define METRIC_VAR_AUTO_LATENCY_DURATION_NS(name) __##name##_auto_latency.duration_ns()
+
 namespace dsn {
 class metric;                  // IWYU pragma: keep
 class metric_entity_prototype; // IWYU pragma: keep
@@ -598,6 +609,7 @@ enum class metric_unit : size_t
     kMilliSeconds,
     kSeconds,
     kRequests,
+    kValues,
     kInvalidUnit,
 };
 
@@ -1409,6 +1421,8 @@ public:
         }
     }
 
+    inline uint64_t duration_ns() const { return _chrono.duration_ns(); }
+
 private:
     percentile_ptr<int64_t> _percentile;
     utils::chronograph _chrono;
diff --git a/src/utils/test/metrics_test.cpp b/src/utils/test/metrics_test.cpp
index 670af36b7..9388a9a92 100644
--- a/src/utils/test/metrics_test.cpp
+++ b/src/utils/test/metrics_test.cpp
@@ -3134,8 +3134,6 @@ void MetricVarTest::test_set_percentile(size_t n, int64_t val)
     EXPECT_EQ(std::vector<int64_t>(n, val), METRIC_VAR_SAMPLES(test_replica_percentile_int64_ns));
 }
 
-#define METRIC_VAR_VALUE(name) _##name->value()
-
 #define TEST_METRIC_VAR_INCREMENT(name)                                                            \
     do {                                                                                           \
         ASSERT_EQ(0, METRIC_VAR_VALUE(name));                                                      \
diff --git a/src/utils/time_utils.h b/src/utils/time_utils.h
index 1db1676b0..265cc3c4d 100644
--- a/src/utils/time_utils.h
+++ b/src/utils/time_utils.h
@@ -142,7 +142,7 @@ public:
 
     inline void reset_start_time() { _start_time_ns = dsn_now_ns(); }
 
-    inline uint64_t duration_ns()
+    inline uint64_t duration_ns() const
     {
         auto now = dsn_now_ns();
         CHECK_GE(now, _start_time_ns);


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


[incubator-pegasus] 21/23: feat(collector): migrate the collector from pegasus-kv/collector (#1461)

Posted by wa...@apache.org.
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 ddd3a2c1257a09b8f1ab8fe83fcb3553c33ec00b
Author: Yingchun Lai <la...@apache.org>
AuthorDate: Thu Apr 27 09:47:02 2023 +0800

    feat(collector): migrate the collector from pegasus-kv/collector (#1461)
    
    Co-authored-by: neverchanje <wu...@xiaomi.com>
---
 .github/workflows/lint_and_test_collector.yml      |  72 ++
 .github/workflows/module_labeler_conf.yml          |   2 +
 .github/workflows/standardization_lint.yaml        |  19 +-
 .gitignore                                         |   3 +
 .../module_labeler_conf.yml => collector/Makefile  |  42 +-
 collector/README.md                                |  28 +
 collector/aggregate/aggregatable.go                |  92 +++
 collector/aggregate/aggregator.go                  | 173 +++++
 collector/aggregate/aggregator_test.go             |  97 +++
 collector/aggregate/history.go                     | 111 ++++
 collector/aggregate/history_test.go                |  40 ++
 collector/aggregate/hook.go                        |  69 ++
 collector/aggregate/hook_test.go                   |  51 ++
 collector/aggregate/perf_client.go                 | 237 +++++++
 collector/aggregate/perf_client_test.go            |  49 ++
 collector/aggregate/perf_counter_decoder.go        |  62 ++
 collector/aggregate/perf_counter_decoder_test.go   |  61 ++
 collector/aggregate/perf_session.go                |  91 +++
 collector/aggregate/table_stats.go                 | 134 ++++
 collector/avail/detector.go                        | 124 ++++
 .../config.yml                                     |  65 +-
 collector/go.mod                                   |  47 ++
 collector/go.sum                                   | 739 +++++++++++++++++++++
 collector/hotspot/algo.go                          |  18 +
 collector/main.go                                  |  99 +++
 collector/metrics/falcon_sink.go                   | 144 ++++
 collector/metrics/prometheus_sink.go               | 110 +++
 collector/metrics/sink.go                          |  53 ++
 collector/templates/index.html                     |  80 +++
 collector/usage/usage_recorder.go                  | 115 ++++
 collector/webui/index.go                           |  84 +++
 collector/webui/tables.go                          |  47 ++
 collector/webui/webserver.go                       |  61 ++
 33 files changed, 3139 insertions(+), 80 deletions(-)

diff --git a/.github/workflows/lint_and_test_collector.yml b/.github/workflows/lint_and_test_collector.yml
new file mode 100644
index 000000000..63909e0ae
--- /dev/null
+++ b/.github/workflows/lint_and_test_collector.yml
@@ -0,0 +1,72 @@
+# 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.
+---
+# workflow name
+name: Golang Lint and Unit Test - collector
+
+# on events
+on:
+  # run on each pull request
+  pull_request:
+    types: [ synchronize, reopened, opened ]
+    branches:
+      - master
+      - 'v[0-9]+.*' # release branch
+      - ci-test # testing branch for github action
+      - '*dev'
+    paths:
+      - collector/**
+
+  # for manually triggering workflow
+  workflow_dispatch:
+
+# workflow tasks
+jobs:
+  lint:
+    name: Lint
+    runs-on: ubuntu-20.04
+    steps:
+      - name: Checkout
+        uses: actions/checkout@v3
+        with:
+          fetch-depth: 1
+      - name: Set up Go
+        uses: actions/setup-go@v4
+        with:
+          go-version: 1.14
+          cache: false
+      - name: Lint
+        uses: golangci/golangci-lint-action@v3
+        with:
+          version: v1.29
+          working-directory: ./collector
+
+  build:
+    name: Build
+    runs-on: ubuntu-20.04
+    steps:
+      - name: Checkout
+        uses: actions/checkout@v3
+        with:
+          fetch-depth: 1
+      - name: Set up Go
+        uses: actions/setup-go@v2
+        with:
+          go-version: 1.14
+      - name: Build
+        working-directory: ./collector
+        run: make
diff --git a/.github/workflows/module_labeler_conf.yml b/.github/workflows/module_labeler_conf.yml
index d5648edf9..ad23e4bf0 100644
--- a/.github/workflows/module_labeler_conf.yml
+++ b/.github/workflows/module_labeler_conf.yml
@@ -19,6 +19,8 @@ github:
   - .github/**/*
 admin-cli:
   - admin-cli/**/*
+collector:
+  - collector/**/*
 docker:
   - docker/**/*
 go-client:
diff --git a/.github/workflows/standardization_lint.yaml b/.github/workflows/standardization_lint.yaml
index ef94b8995..99a4ad836 100644
--- a/.github/workflows/standardization_lint.yaml
+++ b/.github/workflows/standardization_lint.yaml
@@ -62,15 +62,16 @@ jobs:
           bodyRegex: '#(\d+)'
           bodyURLRegex: 'http(s?):\/\/(github.com)(\/apache)(\/incubator-pegasus)(\/issues)\/\d+'
 
-  dockerfile_linter:
-    name: Lint Dockerfile
-    runs-on: ubuntu-latest
-    steps:
-      - uses: actions/checkout@v3
-      - uses: hadolint/hadolint-action@v3.1.0
-        with:
-          recursive: true
-          ignore: 'DL3033,DL3013,DL3059,SC2086,DL3003,SC2164,DL3008,DL3007,DL3006,DL4001'
+# TODO(yingchun): hadolint/hadolint-action@v3.1.0 is not allowed to be used in apache/incubator-pegasus.
+#  dockerfile_linter:
+#    name: Lint Dockerfile
+#    runs-on: ubuntu-latest
+#    steps:
+#      - uses: actions/checkout@v3
+#      - uses: hadolint/hadolint-action@v3.1.0
+#        with:
+#          recursive: true
+#          ignore: 'DL3033,DL3013,DL3059,SC2086,DL3003,SC2164,DL3008,DL3007,DL3006,DL4001'
 
   license_check:
     name: Check License
diff --git a/.gitignore b/.gitignore
index 457dbf309..a3ce7122c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -271,3 +271,6 @@ thirdparty/output/
 
 #macOS
 .DS_Store
+
+#collector
+collector/collector
diff --git a/.github/workflows/module_labeler_conf.yml b/collector/Makefile
similarity index 58%
copy from .github/workflows/module_labeler_conf.yml
copy to collector/Makefile
index d5648edf9..7fab99e04 100644
--- a/.github/workflows/module_labeler_conf.yml
+++ b/collector/Makefile
@@ -14,39 +14,11 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
----
-github:
-  - .github/**/*
-admin-cli:
-  - admin-cli/**/*
-docker:
-  - docker/**/*
-go-client:
-  - go-client/**/*
-java-client:
-  - java-client/**/*
-nodejs-client:
-  - nodejs-client/**/*
-pegic:
-  - pegic/**/*
-python-client:
-  - python-client/**/*
-scala-client:
-  - scala-client/**/*
-thirdparty:
-  - thirdparty/**/*
-thrift:
-  - '**/*.thrift'
-docs:
-  - '**/*.md'
-scripts:
-  - 'scripts/**/*'
-  - '**/*.sh'
+
 build:
-  - 'cmake_modules/**/*'
-  - '**/CMakeLists.txt'
-  - 'run.sh'
-cpp:
-  # TODO(yingchun): add more fine-grained labels
-  - 'src/**/*.h'
-  - 'src/**/*.cpp'
+	go mod tidy
+	go mod verify
+	go build -o collector
+
+fmt:
+	go fmt ./...
diff --git a/collector/README.md b/collector/README.md
new file mode 100644
index 000000000..7f8b09ac2
--- /dev/null
+++ b/collector/README.md
@@ -0,0 +1,28 @@
+<!--
+  ~ 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.
+-->
+
+# Pegasus Collector
+
+[中文文档]
+
+Collector is a part of the Pegasus ecosystem that serves as:
+
+1. the service availability detector
+2. the hotkey detector
+3. the capacity units recorder
diff --git a/collector/aggregate/aggregatable.go b/collector/aggregate/aggregatable.go
new file mode 100644
index 000000000..7ff8c5fb3
--- /dev/null
+++ b/collector/aggregate/aggregatable.go
@@ -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.
+
+package aggregate
+
+var v1Tov2MetricsConversion = map[string]string{
+	"replica*app.pegasus*get_qps":                                  "get_qps",
+	"replica*app.pegasus*multi_get_qps":                            "multi_get_qps",
+	"replica*app.pegasus*put_qps":                                  "put_qps",
+	"replica*app.pegasus*multi_put_qps":                            "multi_put_qps",
+	"replica*app.pegasus*remove_qps":                               "remove_qps",
+	"replica*app.pegasus*multi_remove_qps":                         "multi_remove_qps",
+	"replica*app.pegasus*incr_qps":                                 "incr_qps",
+	"replica*app.pegasus*check_and_set_qps":                        "check_and_set_qps",
+	"replica*app.pegasus*check_and_mutate_qps":                     "check_and_mutate_qps",
+	"replica*app.pegasus*scan_qps":                                 "scan_qps",
+	"replica*eon.replica*backup_request_qps":                       "backup_request_qps",
+	"replica*app.pegasus*duplicate_qps":                            "duplicate_qps",
+	"replica*app.pegasus*dup_shipped_ops":                          "dup_shipped_ops",
+	"replica*app.pegasus*dup_failed_shipping_ops":                  "dup_failed_shipping_ops",
+	"replica*app.pegasus*get_bytes":                                "get_bytes",
+	"replica*app.pegasus*multi_get_bytes":                          "multi_get_bytes",
+	"replica*app.pegasus*scan_bytes":                               "scan_bytes",
+	"replica*app.pegasus*put_bytes":                                "put_bytes",
+	"replica*app.pegasus*multi_put_bytes":                          "multi_put_bytes",
+	"replica*app.pegasus*check_and_set_bytes":                      "check_and_set_bytes",
+	"replica*app.pegasus*check_and_mutate_bytes":                   "check_and_mutate_bytes",
+	"replica*app.pegasus*recent.read.cu":                           "recent_read_cu",
+	"replica*app.pegasus*recent.write.cu":                          "recent_write_cu",
+	"replica*app.pegasus*recent.expire.count":                      "recent_expire_count",
+	"replica*app.pegasus*recent.filter.count":                      "recent_filter_count",
+	"replica*app.pegasus*recent.abnormal.count":                    "recent_abnormal_count",
+	"replica*eon.replica*recent.write.throttling.delay.count":      "recent_write_throttling_delay_count",
+	"replica*eon.replica*recent.write.throttling.reject.count":     "recent_write_throttling_reject_count",
+	"replica*app.pegasus*disk.storage.sst(MB)":                     "sst_storage_mb",
+	"replica*app.pegasus*disk.storage.sst.count":                   "sst_count",
+	"replica*app.pegasus*rdb.block_cache.hit_count":                "rdb_block_cache_hit_count",
+	"replica*app.pegasus*rdb.block_cache.total_count":              "rdb_block_cache_total_count",
+	"replica*app.pegasus*rdb.index_and_filter_blocks.memory_usage": "rdb_index_and_filter_blocks_mem_usage",
+	"replica*app.pegasus*rdb.memtable.memory_usage":                "rdb_memtable_mem_usage",
+	"replica*app.pegasus*rdb.estimate_num_keys":                    "rdb_estimate_num_keys",
+	"replica*app.pegasus*rdb.bf_seek_negatives":                    "rdb_bf_seek_negatives",
+	"replica*app.pegasus*rdb.bf_seek_total":                        "rdb_bf_seek_total",
+	"replica*app.pegasus*rdb.bf_point_positive_true":               "rdb_bf_point_positive_true",
+	"replica*app.pegasus*rdb.bf_point_positive_total":              "rdb_bf_point_positive_total",
+	"replica*app.pegasus*rdb.bf_point_negatives":                   "rdb_bf_point_negatives",
+}
+
+var aggregatableSet = map[string]interface{}{
+	"read_qps":    nil,
+	"write_qps":   nil,
+	"read_bytes":  nil,
+	"write_bytes": nil,
+}
+
+// aggregatable returns whether the counter is to be aggregated on collector,
+// including v1Tov2MetricsConversion and aggregatableSet.
+func aggregatable(pc *partitionPerfCounter) bool {
+	v2Name, found := v1Tov2MetricsConversion[pc.name]
+	if found { // ignored
+		pc.name = v2Name
+		return true // listed above are all aggregatable
+	}
+	_, found = aggregatableSet[pc.name]
+	return found
+}
+
+// AllMetrics returns metrics tracked within this collector.
+// The sets of metrics from cluster level and table level are completely equal.
+func AllMetrics() (res []string) {
+	for _, newName := range v1Tov2MetricsConversion {
+		res = append(res, newName)
+	}
+	for name := range aggregatableSet {
+		res = append(res, name)
+	}
+	return res
+}
diff --git a/collector/aggregate/aggregator.go b/collector/aggregate/aggregator.go
new file mode 100644
index 000000000..7e0cf50b2
--- /dev/null
+++ b/collector/aggregate/aggregator.go
@@ -0,0 +1,173 @@
+// 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.
+
+package aggregate
+
+import (
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+	"gopkg.in/tomb.v2"
+)
+
+// TableStatsAggregator aggregates the metric on each partition into table-level metrics.
+// It's reponsible for all tables in the pegasus cluster.
+// After all TableStats have been collected, TableStatsAggregator sums them up into a
+// ClusterStats. Users of this pacakage can use the hooks to watch every changes of the stats.
+type TableStatsAggregator interface {
+	Aggregate() (map[int32]*TableStats, *ClusterStats, error)
+
+	Close()
+}
+
+// NewTableStatsAggregator returns a TableStatsAggregator instance.
+func NewTableStatsAggregator(metaAddrs []string) TableStatsAggregator {
+	return &tableStatsAggregator{
+		tables: make(map[int32]*TableStats),
+		client: NewPerfClient(metaAddrs),
+	}
+}
+
+type tableStatsAggregator struct {
+	tables   map[int32]*TableStats
+	allStats *ClusterStats
+
+	client *PerfClient
+}
+
+// Start looping for metrics aggregation
+func Start(tom *tomb.Tomb) {
+	aggregateInterval := viper.GetDuration("metrics.report_interval")
+	ticker := time.NewTicker(aggregateInterval)
+
+	metaAddr := viper.GetString("meta_server")
+	iAg := NewTableStatsAggregator([]string{metaAddr})
+	ag := iAg.(*tableStatsAggregator)
+
+	for {
+		select {
+		case <-tom.Dying(): // check if context cancelled
+			return
+		case <-ticker.C:
+		}
+
+		_, _, err := ag.Aggregate()
+		if err != nil {
+			log.Error(err)
+		}
+
+		// produce stats for the hooks
+		var batchTableStats []TableStats
+		for _, table := range ag.tables {
+			batchTableStats = append(batchTableStats, *table)
+		}
+		ag.aggregateClusterStats()
+		hooksManager.afterTableStatsEmitted(batchTableStats, *ag.allStats)
+	}
+}
+
+func (ag *tableStatsAggregator) Aggregate() (map[int32]*TableStats, *ClusterStats, error) {
+	err := ag.updateTableMap()
+	if err != nil {
+		return nil, nil, fmt.Errorf("failed to aggregate: %s", err)
+	}
+
+	// TODO(wutao1): reduce meta queries for listing nodes
+	partitions, err := ag.client.GetPartitionStats()
+	if err != nil {
+		return nil, nil, fmt.Errorf("failed to aggregate: %s", err)
+	}
+	for _, p := range partitions {
+		ag.updatePartitionStat(p)
+	}
+
+	for _, table := range ag.tables {
+		table.aggregate()
+	}
+
+	return ag.tables, ag.allStats, nil
+}
+
+func (ag *tableStatsAggregator) Close() {
+	ag.client.Close()
+}
+
+func (ag *tableStatsAggregator) aggregateClusterStats() {
+	ag.allStats = &ClusterStats{
+		Stats:     make(map[string]float64),
+		Timestamp: time.Now(),
+	}
+	for _, table := range ag.tables {
+		for k, v := range table.Stats {
+			ag.allStats.Stats[k] += v
+		}
+	}
+}
+
+// Some tables may disappear (be dropped) or first show up.
+// This function maintains the local table map
+// to keep consistent with the pegasus cluster.
+func (ag *tableStatsAggregator) updateTableMap() error {
+	tables, err := ag.client.listTables()
+	if err != nil {
+		return err
+	}
+	ag.doUpdateTableMap(tables)
+	return nil
+}
+
+func (ag *tableStatsAggregator) doUpdateTableMap(tables []*admin.AppInfo) {
+	currentTableSet := make(map[int32]*struct{})
+	for _, tb := range tables {
+		currentTableSet[tb.AppID] = nil
+		if _, found := ag.tables[tb.AppID]; !found {
+			// non-exisistent table, create it
+			ag.tables[tb.AppID] = newTableStats(tb)
+			log.Infof("found new table: %+v", tb)
+
+			// TODO(wutao1): some tables may have partitions splitted,
+			//               recreate the tableStats then.
+		}
+	}
+	for appID, tb := range ag.tables {
+		// disappeared table, delete it
+		if _, found := currentTableSet[appID]; !found {
+			log.Infof("remove table from collector: {AppID: %d, PartitionCount: %d}", appID, len(tb.Partitions))
+			delete(ag.tables, appID)
+
+			hooksManager.afterTableDropped(appID)
+		}
+	}
+}
+
+// Update the counter value.
+func (ag *tableStatsAggregator) updatePartitionStat(pc *PartitionStats) {
+	tb, found := ag.tables[pc.Gpid.Appid]
+	if !found {
+		// Ignore the perf-counter because there's currently no such table
+		return
+	}
+	part, found := tb.Partitions[int(pc.Gpid.PartitionIndex)]
+	if !found {
+		log.Errorf("no such partition %+v", pc.Gpid)
+		return
+	}
+	*part = *pc
+}
diff --git a/collector/aggregate/aggregator_test.go b/collector/aggregate/aggregator_test.go
new file mode 100644
index 000000000..4b73e9213
--- /dev/null
+++ b/collector/aggregate/aggregator_test.go
@@ -0,0 +1,97 @@
+// 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.
+
+package aggregate
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestUpdateLocalTableMap(t *testing.T) {
+	ag := &tableStatsAggregator{
+		client: NewPerfClient([]string{"127.0.0.1:34601"}),
+		tables: make(map[int32]*TableStats),
+	}
+	err := ag.updateTableMap()
+	assert.Nil(t, err)
+	assert.Equal(t, len(ag.tables), 2)
+	assert.Equal(t, len(ag.tables[1].Partitions), 4) // test
+	assert.Equal(t, len(ag.tables[2].Partitions), 8) // stat
+
+	tables := []*admin.AppInfo{
+		{AppID: 1, AppName: "stat", PartitionCount: 4},
+		{AppID: 2, AppName: "test", PartitionCount: 8},
+		{AppID: 3, AppName: "new_table", PartitionCount: 16},
+	}
+	ag.doUpdateTableMap(tables)
+	assert.Equal(t, len(ag.tables), 3)
+	assert.Equal(t, len(ag.tables[3].Partitions), 16)
+
+	tables = []*admin.AppInfo{
+		{AppID: 1, AppName: "stat", PartitionCount: 4},
+	}
+	ag.doUpdateTableMap(tables)
+	assert.Equal(t, len(ag.tables), 1)
+	assert.Equal(t, len(ag.tables[1].Partitions), 4)
+}
+
+func TestUpdatePartitionStats(t *testing.T) {
+	ag := &tableStatsAggregator{
+		tables: make(map[int32]*TableStats),
+	}
+	tables := []*admin.AppInfo{
+		{AppID: 1, AppName: "stat", PartitionCount: 4},
+	}
+	ag.doUpdateTableMap(tables)
+
+	pc := decodePartitionPerfCounter("replica*app.pegasus*recent.abnormal.count@1.2", 100)
+	assert.NotNil(t, pc)
+
+	ag.updatePartitionStat(&PartitionStats{
+		Gpid: base.Gpid{Appid: 1, PartitionIndex: 2},
+		Addr: "127.0.0.1:34601",
+		Stats: map[string]float64{
+			"replica*app.pegasus*recent.abnormal.count": 100,
+		},
+	})
+
+	part := ag.tables[1].Partitions[2]
+	assert.Contains(t, part.Stats, pc.name)
+	assert.Equal(t, part.Stats[pc.name], float64(100))
+	assert.Equal(t, part.Addr, "127.0.0.1:34601")
+}
+
+func TestAggregate(t *testing.T) {
+	ag := NewTableStatsAggregator([]string{"127.0.0.1:34601"})
+	tableStats, allStat, err := ag.Aggregate()
+	assert.Nil(t, err)
+	assert.Greater(t, len(allStat.Stats), 0)
+
+	assert.Equal(t, len(tableStats), 2)
+
+	// ensure len(tableStats) == len(partitionStats) == len(clusterStats)
+	for _, tb := range tableStats {
+		assert.Equal(t, len(tb.Stats), len(allStat.Stats))
+		for _, p := range tb.Partitions {
+			assert.Equal(t, len(p.Stats), len(allStat.Stats))
+		}
+	}
+}
diff --git a/collector/aggregate/history.go b/collector/aggregate/history.go
new file mode 100644
index 000000000..27e6d10ee
--- /dev/null
+++ b/collector/aggregate/history.go
@@ -0,0 +1,111 @@
+// 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.
+
+package aggregate
+
+import (
+	"container/list"
+	"sync"
+)
+
+const (
+	historyMaxCapacity = 10
+)
+
+// threadSafeHistory is a time-ordered queue of stats.
+type threadSafeHistory struct {
+	lock sync.RWMutex
+
+	stats    *list.List
+	capacity int
+}
+
+// Emit a TableStats to the history. Will remove the oldest record from history.
+func (h *threadSafeHistory) emit(stat interface{}) {
+	h.lock.Lock()
+	defer h.lock.Unlock()
+
+	if h.stats.Len() == h.capacity {
+		h.stats.Remove(h.stats.Front())
+	}
+	h.stats.PushBack(stat)
+}
+
+func newHistory(capacity int) *threadSafeHistory {
+	return &threadSafeHistory{
+		stats:    list.New(),
+		capacity: capacity,
+	}
+}
+
+type historyStore struct {
+	lock sync.RWMutex
+
+	tables  map[int]*threadSafeHistory
+	cluster *threadSafeHistory
+}
+
+var globalHistoryStore = &historyStore{
+	tables:  make(map[int]*threadSafeHistory),
+	cluster: newHistory(historyMaxCapacity),
+}
+
+// SnapshotClusterStats takes a snapshot from the history. The returned array is ordered by time.
+func SnapshotClusterStats() []ClusterStats {
+	s := globalHistoryStore
+
+	s.lock.RLock()
+	defer s.lock.RUnlock()
+
+	var result []ClusterStats
+	l := s.cluster.stats
+	for e := l.Front(); e != nil; e = e.Next() {
+		stat, _ := e.Value.(*ClusterStats)
+		result = append(result, *stat)
+	}
+	return result
+}
+
+func init() {
+	initHistoryStore()
+}
+
+func initHistoryStore() {
+	AddHookAfterTableStatEmitted(func(stats []TableStats, allStat ClusterStats) {
+		s := globalHistoryStore
+
+		s.lock.Lock()
+		defer s.lock.Unlock()
+		for _, stat := range stats {
+			history, found := s.tables[stat.AppID]
+			if !found {
+				history = newHistory(historyMaxCapacity)
+				s.tables[stat.AppID] = history
+			}
+			history.emit(&stat)
+		}
+		s.cluster.emit(&allStat)
+	})
+
+	AddHookAfterTableDropped(func(appID int) {
+		s := globalHistoryStore
+
+		s.lock.Lock()
+		defer s.lock.Unlock()
+		delete(s.tables, appID)
+	})
+}
diff --git a/collector/aggregate/history_test.go b/collector/aggregate/history_test.go
new file mode 100644
index 000000000..e4bbbfa06
--- /dev/null
+++ b/collector/aggregate/history_test.go
@@ -0,0 +1,40 @@
+// 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.
+
+package aggregate
+
+import (
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestHistory(t *testing.T) {
+	hooksManager = tableStatsHooksManager{}
+	initHistoryStore()
+
+	for i := 0; i < historyMaxCapacity*2; i++ {
+		hooksManager.afterTableStatsEmitted([]TableStats{},
+			ClusterStats{Stats: map[string]float64{"write": 100.0 * float64(i)}, Timestamp: time.Now()})
+	}
+	clusterStats := SnapshotClusterStats()
+	assert.Equal(t, len(clusterStats), historyMaxCapacity)
+	for i := 0; i < historyMaxCapacity; i++ {
+		assert.Equal(t, clusterStats[i].Stats["write"], float64(historyMaxCapacity+i)*100.0)
+	}
+}
diff --git a/collector/aggregate/hook.go b/collector/aggregate/hook.go
new file mode 100644
index 000000000..e20af348a
--- /dev/null
+++ b/collector/aggregate/hook.go
@@ -0,0 +1,69 @@
+// 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.
+
+package aggregate
+
+import "sync"
+
+// HookAfterTableStatEmitted is a hook of event that new TableStats are generated.
+// Each call of the hook handles a batch of tables.
+type HookAfterTableStatEmitted func(stats []TableStats, allStats ClusterStats)
+
+// AddHookAfterTableStatEmitted adds a hook of event that a new TableStats is generated.
+func AddHookAfterTableStatEmitted(hk HookAfterTableStatEmitted) {
+	m := &hooksManager
+	m.lock.Lock()
+	defer m.lock.Unlock()
+	m.emittedHooks = append(m.emittedHooks, hk)
+}
+
+// HookAfterTableDropped is a hook of event that a table is dropped.
+type HookAfterTableDropped func(appID int)
+
+// AddHookAfterTableDropped adds a hook of event that a table is dropped.
+func AddHookAfterTableDropped(hk HookAfterTableDropped) {
+	m := &hooksManager
+	m.lock.Lock()
+	defer m.lock.Unlock()
+	m.droppedHooks = append(m.droppedHooks, hk)
+}
+
+type tableStatsHooksManager struct {
+	lock         sync.RWMutex
+	emittedHooks []HookAfterTableStatEmitted
+	droppedHooks []HookAfterTableDropped
+}
+
+func (m *tableStatsHooksManager) afterTableStatsEmitted(stats []TableStats, allStat ClusterStats) {
+	m.lock.RLock()
+	defer m.lock.RUnlock()
+
+	for _, hook := range m.emittedHooks {
+		hook(stats, allStat)
+	}
+}
+
+func (m *tableStatsHooksManager) afterTableDropped(appID int32) {
+	m.lock.RLock()
+	defer m.lock.RUnlock()
+
+	for _, hook := range m.droppedHooks {
+		hook(int(appID))
+	}
+}
+
+var hooksManager tableStatsHooksManager
diff --git a/collector/aggregate/hook_test.go b/collector/aggregate/hook_test.go
new file mode 100644
index 000000000..a1a853314
--- /dev/null
+++ b/collector/aggregate/hook_test.go
@@ -0,0 +1,51 @@
+// 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.
+
+package aggregate
+
+import (
+	"testing"
+	"time"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestAddHook(t *testing.T) {
+	var actualTableStats []TableStats
+	var actualClusterStats ClusterStats
+	AddHookAfterTableStatEmitted(func(stats []TableStats, allStats ClusterStats) {
+		actualTableStats = stats
+		actualClusterStats = allStats
+	})
+
+	t1 := TableStats{TableName: "test", Stats: map[string]float64{"write": 512.0}, Timestamp: time.Now()}
+	t2 := TableStats{TableName: "stat", Stats: map[string]float64{"write": 256.0}, Timestamp: time.Now()}
+	all := ClusterStats{Stats: map[string]float64{"write": 768.0}}
+
+	ch := make(chan interface{})
+	go func() {
+		hooksManager.afterTableStatsEmitted([]TableStats{t1, t2}, all)
+		ch <- nil
+	}()
+	<-ch
+
+	assert.EqualValues(t, actualTableStats, []TableStats{t1, t2})
+	assert.EqualValues(t, actualClusterStats, all)
+
+	// clear up
+	hooksManager = tableStatsHooksManager{}
+}
diff --git a/collector/aggregate/perf_client.go b/collector/aggregate/perf_client.go
new file mode 100644
index 000000000..26739c116
--- /dev/null
+++ b/collector/aggregate/perf_client.go
@@ -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.
+
+package aggregate
+
+import (
+	"context"
+	"fmt"
+	"sync"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/apache/incubator-pegasus/go-client/session"
+	log "github.com/sirupsen/logrus"
+	batchErr "k8s.io/apimachinery/pkg/util/errors"
+)
+
+// PerfClient manages sessions to all replica nodes.
+type PerfClient struct {
+	meta *session.MetaManager
+
+	nodes map[string]*PerfSession
+}
+
+// GetPartitionStats retrieves all the partition stats from replica nodes.
+// NOTE: Only the primaries are counted.
+func (m *PerfClient) GetPartitionStats() ([]*PartitionStats, error) {
+	m.updateNodes()
+
+	partitions, err := m.preparePrimariesStats()
+	if err != nil {
+		return nil, err
+	}
+
+	nodeStats, err := m.GetNodeStats("@")
+	if err != nil {
+		return nil, err
+	}
+
+	for _, n := range nodeStats {
+		for name, value := range n.Stats {
+			perfCounter := decodePartitionPerfCounter(name, value)
+			if perfCounter == nil {
+				continue
+			}
+			if !aggregatable(perfCounter) {
+				continue
+			}
+			part := partitions[perfCounter.gpid]
+			if part == nil || part.Addr != n.Addr {
+				// if this node is not the primary of this partition
+				continue
+			}
+
+			part.Stats[perfCounter.name] = perfCounter.value
+		}
+	}
+
+	var ret []*PartitionStats
+	for _, part := range partitions {
+		extendStats(&part.Stats)
+		ret = append(ret, part)
+	}
+	return ret, nil
+}
+
+// getPrimaries returns mapping of [partition -> primary address]
+func (m *PerfClient) getPrimaries() (map[base.Gpid]string, error) {
+	tables, err := m.listTables()
+	if err != nil {
+		return nil, err
+	}
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
+	defer cancel()
+
+	result := make(map[base.Gpid]string)
+	var mu sync.Mutex
+	var funcs []func() error
+
+	for _, table := range tables {
+		tb := table
+		funcs = append(funcs, func() (subErr error) {
+			tableCfg, err := m.meta.QueryConfig(ctx, tb.AppName)
+			if err != nil {
+				return fmt.Errorf("failed on table(%s): %s", tb.AppName, err)
+			}
+			mu.Lock()
+			for _, p := range tableCfg.Partitions {
+				result[*p.Pid] = p.Primary.GetAddress()
+			}
+			mu.Unlock()
+			return nil
+		})
+	}
+	return result, batchErr.AggregateGoroutines(funcs...)
+}
+
+func (m *PerfClient) preparePrimariesStats() (map[base.Gpid]*PartitionStats, error) {
+	primaries, err := m.getPrimaries()
+	if err != nil {
+		return nil, err
+	}
+	partitions := make(map[base.Gpid]*PartitionStats)
+	for p, addr := range primaries {
+		partitions[p] = &PartitionStats{
+			Gpid:  p,
+			Stats: make(map[string]float64),
+			Addr:  addr,
+		}
+	}
+	return partitions, nil
+}
+
+// NodeStat contains the stats of a replica node.
+type NodeStat struct {
+	// Address of the replica node.
+	Addr string
+
+	// perfCounter's name -> the value.
+	Stats map[string]float64
+}
+
+// GetNodeStats retrieves all the stats matched with `filter` from replica nodes.
+func (m *PerfClient) GetNodeStats(filter string) ([]*NodeStat, error) {
+	m.updateNodes()
+
+	// concurrently send RPC for perf-counters.
+	var results []*NodeStat
+	var funcs []func() error
+	var mu sync.Mutex
+
+	for _, node := range m.nodes {
+		n := node
+		funcs = append(funcs, func() (subErr error) {
+			stat := &NodeStat{
+				Addr:  n.Address,
+				Stats: make(map[string]float64),
+			}
+			perfCounters, err := n.GetPerfCounters(filter)
+			if err != nil {
+				return fmt.Errorf("failed on node(%s): %s", n.Address, err)
+			}
+			for _, p := range perfCounters {
+				stat.Stats[p.Name] = p.Value
+			}
+
+			mu.Lock()
+			results = append(results, stat)
+			defer mu.Unlock()
+			return nil
+		})
+	}
+	return results, batchErr.AggregateGoroutines(funcs...)
+}
+
+func (m *PerfClient) listNodes() ([]*admin.NodeInfo, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
+	defer cancel()
+	resp, err := m.meta.ListNodes(ctx, &admin.ListNodesRequest{
+		Status: admin.NodeStatus_NS_ALIVE,
+	})
+	if err != nil {
+		return nil, err
+	}
+	return resp.Infos, nil
+}
+
+func (m *PerfClient) listTables() ([]*admin.AppInfo, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
+	defer cancel()
+	resp, err := m.meta.ListApps(ctx, &admin.ListAppsRequest{
+		Status: admin.AppStatus_AS_AVAILABLE,
+	})
+	if err != nil {
+		return nil, err
+	}
+	return resp.Infos, nil
+}
+
+// updateNodes
+func (m *PerfClient) updateNodes() {
+	nodeInfos, err := m.listNodes()
+	if err != nil {
+		log.Error("skip updating nodes due to list-nodes RPC failure: ", err)
+		return
+	}
+
+	newNodes := make(map[string]*PerfSession)
+	for _, n := range nodeInfos {
+		addr := n.Address.GetAddress()
+		node, found := m.nodes[addr]
+		if !found {
+			newNodes[addr] = NewPerfSession(addr)
+		} else {
+			newNodes[addr] = node
+		}
+	}
+	for n, client := range m.nodes {
+		// close the unused connections
+		if _, found := newNodes[n]; !found {
+			client.Close()
+		}
+	}
+	m.nodes = newNodes
+}
+
+// Close release the resources.
+func (m *PerfClient) Close() {
+	m.meta.Close()
+
+	for _, n := range m.nodes {
+		n.Close()
+	}
+}
+
+// NewPerfClient returns an instance of PerfClient.
+func NewPerfClient(metaAddrs []string) *PerfClient {
+	return &PerfClient{
+		meta:  session.NewMetaManager(metaAddrs, session.NewNodeSession),
+		nodes: make(map[string]*PerfSession),
+	}
+}
diff --git a/collector/aggregate/perf_client_test.go b/collector/aggregate/perf_client_test.go
new file mode 100644
index 000000000..264eb60b3
--- /dev/null
+++ b/collector/aggregate/perf_client_test.go
@@ -0,0 +1,49 @@
+// 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.
+
+package aggregate
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestPerfClientGetNodeStats(t *testing.T) {
+	pclient := NewPerfClient([]string{"127.0.0.1:34601"})
+	nodes, err := pclient.GetNodeStats("@")
+	assert.Nil(t, err)
+	assert.Greater(t, len(nodes), 0)
+	assert.Greater(t, len(nodes[0].Stats), 0)
+	for _, n := range nodes {
+		assert.NotEmpty(t, n.Addr)
+	}
+}
+
+func TestPerfClientGetPartitionStats(t *testing.T) {
+	pclient := NewPerfClient([]string{"127.0.0.1:34601"})
+	partitions, err := pclient.GetPartitionStats()
+	assert.Nil(t, err)
+	assert.Greater(t, len(partitions), 0)
+	assert.Greater(t, len(partitions[0].Stats), 0)
+	for _, p := range partitions {
+		assert.NotEmpty(t, p.Addr)
+		assert.NotEqual(t, p.Gpid, base.Gpid{Appid: 0, PartitionIndex: 0})
+		assert.NotEmpty(t, p.Stats)
+	}
+}
diff --git a/collector/aggregate/perf_counter_decoder.go b/collector/aggregate/perf_counter_decoder.go
new file mode 100644
index 000000000..f89e31840
--- /dev/null
+++ b/collector/aggregate/perf_counter_decoder.go
@@ -0,0 +1,62 @@
+// 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.
+
+package aggregate
+
+import (
+	"strconv"
+	"strings"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+)
+
+type partitionPerfCounter struct {
+	name  string
+	gpid  base.Gpid
+	value float64
+}
+
+// decodePartitionPerfCounter implements the v1 version of metric decoding.
+func decodePartitionPerfCounter(name string, value float64) *partitionPerfCounter {
+	idx := strings.LastIndex(name, "@")
+	gpidStr := name[idx+1:]
+	appIDAndPartitionID := strings.Split(gpidStr, ".")
+	if len(appIDAndPartitionID) != 2 {
+		// special case: in some mis-desgined metrics, what follows after a '@' may not be a replica id
+		return nil
+	}
+	appIDAndPartitionID = appIDAndPartitionID[:2] // "AppID.PartitionIndex"
+	appID, err := strconv.Atoi(appIDAndPartitionID[0])
+	if err != nil {
+		return nil
+	}
+	partitionIndex, err := strconv.Atoi(appIDAndPartitionID[1])
+	if err != nil {
+		return nil
+	}
+	return &partitionPerfCounter{
+		name: name[:idx], // strip out the replica id
+		gpid: base.Gpid{
+			Appid:          int32(appID),
+			PartitionIndex: int32(partitionIndex),
+		},
+		value: value,
+	}
+}
+
+// TODO(wutao1): implement the v2 version of metric decoding according to
+// https://github.com/apache/incubator-pegasus/blob/master/rfcs/2020-08-27-metric-api.md
diff --git a/collector/aggregate/perf_counter_decoder_test.go b/collector/aggregate/perf_counter_decoder_test.go
new file mode 100644
index 000000000..e910cd9f8
--- /dev/null
+++ b/collector/aggregate/perf_counter_decoder_test.go
@@ -0,0 +1,61 @@
+// 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.
+
+package aggregate
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/stretchr/testify/assert"
+)
+
+func TestDecodePartitionPerfCounter(t *testing.T) {
+	tests := []struct {
+		name string
+
+		isNil          bool
+		counterName    string
+		appID          int32
+		partitionIndex int32
+	}{
+		{name: "replica*app.pegasus*get_latency@2.5.p999", isNil: true},
+
+		// server-level counter, does not contain gpid.
+		{name: "replica*eon.replica*table.level.RPC_RRDB_RRDB_CHECK_AND_MUTATE.latency(ns)@temp", isNil: true},
+		{
+			name:  "replica*eon.replica*table.level.RPC_RRDB_RRDB_MULTI_PUT.latency(ns)@temp.p999",
+			isNil: true,
+		},
+
+		{
+			name:           "replica*app.pegasus*recent.abnormal.count@1.2",
+			counterName:    "replica*app.pegasus*recent.abnormal.count",
+			appID:          1,
+			partitionIndex: 2,
+		},
+	}
+
+	for _, tt := range tests {
+		pc := decodePartitionPerfCounter(tt.name, 1.0)
+		assert.Equal(t, pc == nil, tt.isNil, tt.name)
+		if pc != nil {
+			assert.Equal(t, pc.name, tt.counterName)
+			assert.Equal(t, pc.gpid, base.Gpid{Appid: tt.appID, PartitionIndex: tt.partitionIndex})
+		}
+	}
+}
diff --git a/collector/aggregate/perf_session.go b/collector/aggregate/perf_session.go
new file mode 100644
index 000000000..9e0665ac6
--- /dev/null
+++ b/collector/aggregate/perf_session.go
@@ -0,0 +1,91 @@
+// 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.
+
+package aggregate
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/admin"
+	"github.com/apache/incubator-pegasus/go-client/session"
+	"github.com/tidwall/gjson"
+)
+
+// PerfSession is a client to get perf-counters from a Pegasus ReplicaServer.
+type PerfSession struct {
+	session.NodeSession
+
+	Address string
+}
+
+// PerfCounter is a Pegasus perf-counter.
+type PerfCounter struct {
+	Name  string
+	Value float64
+}
+
+func (p *PerfCounter) String() string {
+	return fmt.Sprintf("{Name: %s, Value: %f}", p.Name, p.Value)
+}
+
+// NewPerfSession returns an instance of PerfSession.
+func NewPerfSession(addr string) *PerfSession {
+	return &PerfSession{
+		Address:     addr,
+		NodeSession: session.NewNodeSession(addr, session.NodeTypeReplica),
+	}
+}
+
+// WrapPerf returns an instance of PerfSession using an existed session.
+func WrapPerf(addr string, session session.NodeSession) *PerfSession {
+	return &PerfSession{
+		Address:     addr,
+		NodeSession: session,
+	}
+}
+
+// GetPerfCounters retrieves all perf-counters matched with `filter` from the remote node.
+func (c *PerfSession) GetPerfCounters(filter string) ([]*PerfCounter, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
+	defer cancel()
+	rc := &admin.RemoteCommand{
+		Command:   "perf-counters-by-substr",
+		Arguments: []string{filter},
+	}
+
+	result, err := rc.Call(ctx, c.NodeSession)
+	if err != nil {
+		return nil, err
+	}
+	resultJSON := gjson.Parse(result)
+	perfCounters := resultJSON.Get("counters").Array()
+	var ret []*PerfCounter
+	for _, perfCounter := range perfCounters {
+		ret = append(ret, &PerfCounter{
+			Name:  perfCounter.Get("name").String(),
+			Value: perfCounter.Get("value").Float(),
+		})
+	}
+	return ret, nil
+}
+
+// Close terminates the session to replica.
+func (c *PerfSession) Close() {
+	c.NodeSession.Close()
+}
diff --git a/collector/aggregate/table_stats.go b/collector/aggregate/table_stats.go
new file mode 100644
index 000000000..62d9b7d10
--- /dev/null
+++ b/collector/aggregate/table_stats.go
@@ -0,0 +1,134 @@
+// 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.
+
+package aggregate
+
+import (
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/idl/admin"
+	"github.com/apache/incubator-pegasus/go-client/idl/base"
+)
+
+// PartitionStats is a set of metrics retrieved from this partition.
+type PartitionStats struct {
+	Gpid base.Gpid
+
+	// Address of the primary where this partition locates.
+	Addr string
+
+	// perfCounter's name -> the value.
+	Stats map[string]float64
+}
+
+// TableStats has the aggregated metrics for this table.
+type TableStats struct {
+	TableName string
+	AppID     int
+
+	Partitions map[int]*PartitionStats
+
+	// the time when the stats was generated
+	Timestamp time.Time
+
+	// The aggregated value of table metrics.
+	// perfCounter's name -> the value.
+	Stats map[string]float64
+}
+
+// ClusterStats is the aggregated metrics for all the TableStats in this cluster.
+// For example, 3 tables with "write_qps" [25, 70, 100] are summed up to
+// `Stats: {"write_qps" : 195}`.
+type ClusterStats struct {
+	Timestamp time.Time
+
+	Stats map[string]float64
+}
+
+func newTableStats(info *admin.AppInfo) *TableStats {
+	tb := &TableStats{
+		TableName:  info.AppName,
+		AppID:      int(info.AppID),
+		Partitions: make(map[int]*PartitionStats),
+		Stats:      make(map[string]float64),
+		Timestamp:  time.Now(),
+	}
+	for i := 0; i < int(info.PartitionCount); i++ {
+		tb.Partitions[i] = &PartitionStats{
+			Gpid:  base.Gpid{Appid: int32(info.AppID), PartitionIndex: int32(i)},
+			Stats: make(map[string]float64),
+		}
+	}
+	return tb
+}
+
+func (tb *TableStats) aggregate() {
+	tb.Timestamp = time.Now()
+	for _, part := range tb.Partitions {
+		for name, value := range part.Stats {
+			tb.Stats[name] += value
+		}
+	}
+}
+
+func aggregateCustomStats(elements []string, stats *map[string]float64, resultName string) {
+	aggregated := float64(0)
+	for _, ele := range elements {
+		if v, found := (*stats)[ele]; found {
+			aggregated += v
+		}
+	}
+	(*stats)[resultName] = aggregated
+}
+
+// Extends the stat with read_qps/read_bytes/write_qps/write_bytes.
+func extendStats(stats *map[string]float64) {
+	var reads = []string{
+		"get",
+		"multi_get",
+		"scan",
+	}
+	var readQPS []string
+	for _, r := range reads {
+		readQPS = append(readQPS, r+"_qps")
+	}
+	var readBytes []string
+	for _, r := range reads {
+		readBytes = append(readBytes, r+"_bytes")
+	}
+	aggregateCustomStats(readQPS, stats, "read_qps")
+	aggregateCustomStats(readBytes, stats, "read_bytes")
+
+	var writes = []string{
+		"put",
+		"remove",
+		"multi_put",
+		"multi_remove",
+		"check_and_set",
+		"check_and_mutate",
+	}
+	var writeQPS []string
+	for _, w := range writes {
+		writeQPS = append(writeQPS, w+"_qps")
+	}
+	var writeBytes []string
+	for _, w := range writes {
+		writeBytes = append(writeBytes, w+"_bytes")
+	}
+	aggregateCustomStats(writeQPS, stats, "write_qps")
+	aggregateCustomStats(writeBytes, stats, "write_bytes")
+}
diff --git a/collector/avail/detector.go b/collector/avail/detector.go
new file mode 100644
index 000000000..1e23a6210
--- /dev/null
+++ b/collector/avail/detector.go
@@ -0,0 +1,124 @@
+// 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.
+
+package avail
+
+import (
+	"context"
+	"sync/atomic"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/pegasus"
+	log "github.com/sirupsen/logrus"
+)
+
+// Detector periodically checks the service availability of the Pegasus cluster.
+type Detector interface {
+
+	// Start detection until the ctx cancelled. This method will block the current thread.
+	Start(ctx context.Context) error
+}
+
+// NewDetector returns a service-availability detector.
+func NewDetector(client pegasus.Client) Detector {
+	return &pegasusDetector{client: client}
+}
+
+type pegasusDetector struct {
+	// client reads and writes periodically to a specified table.
+	client      pegasus.Client
+	detectTable pegasus.TableConnector
+
+	detectInterval  time.Duration
+	detectTableName string
+
+	// timeout of a single detect
+	detectTimeout time.Duration
+
+	detectHashKeys [][]byte
+
+	recentMinuteDetectTimes  uint64
+	recentMinuteFailureTimes uint64
+
+	recentHourDetectTimes  uint64
+	recentHourFailureTimes uint64
+
+	recentDayDetectTimes  uint64
+	recentDayFailureTimes uint64
+}
+
+func (d *pegasusDetector) Start(rootCtx context.Context) error {
+	var err error
+	ctx, cancel := context.WithTimeout(rootCtx, 10*time.Second)
+	defer cancel()
+	d.detectTable, err = d.client.OpenTable(ctx, d.detectTableName)
+	if err != nil {
+		return err
+	}
+
+	ticker := time.NewTicker(d.detectInterval)
+	for {
+		select {
+		case <-rootCtx.Done(): // check if context cancelled
+			return nil
+		case <-ticker.C:
+			return nil
+		default:
+		}
+
+		// periodically set/get a configured Pegasus table.
+		d.detect(ctx)
+	}
+}
+
+func (d *pegasusDetector) detect(rootCtx context.Context) {
+	// TODO(yingchun): doesn't work, just to mute lint errors.
+	d.detectPartition(rootCtx, 1)
+}
+
+func (d *pegasusDetector) detectPartition(rootCtx context.Context, partitionIdx int) {
+	d.incrDetectTimes()
+
+	go func() {
+		ctx, cancel := context.WithTimeout(rootCtx, d.detectTimeout)
+		defer cancel()
+
+		hashkey := d.detectHashKeys[partitionIdx]
+		value := []byte("")
+
+		if err := d.detectTable.Set(ctx, hashkey, []byte(""), value); err != nil {
+			d.incrFailureTimes()
+			log.Errorf("set partition [%d] failed, hashkey=\"%s\": %s", partitionIdx, hashkey, err)
+		}
+		if _, err := d.detectTable.Get(ctx, hashkey, []byte("")); err != nil {
+			d.incrFailureTimes()
+			log.Errorf("get partition [%d] failed, hashkey=\"%s\": %s", partitionIdx, hashkey, err)
+		}
+	}()
+}
+
+func (d *pegasusDetector) incrDetectTimes() {
+	atomic.AddUint64(&d.recentMinuteDetectTimes, 1)
+	atomic.AddUint64(&d.recentHourDetectTimes, 1)
+	atomic.AddUint64(&d.recentDayDetectTimes, 1)
+}
+
+func (d *pegasusDetector) incrFailureTimes() {
+	atomic.AddUint64(&d.recentMinuteFailureTimes, 1)
+	atomic.AddUint64(&d.recentHourFailureTimes, 1)
+	atomic.AddUint64(&d.recentDayFailureTimes, 1)
+}
diff --git a/.github/workflows/module_labeler_conf.yml b/collector/config.yml
similarity index 58%
copy from .github/workflows/module_labeler_conf.yml
copy to collector/config.yml
index d5648edf9..b89993d6c 100644
--- a/.github/workflows/module_labeler_conf.yml
+++ b/collector/config.yml
@@ -14,39 +14,32 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
----
-github:
-  - .github/**/*
-admin-cli:
-  - admin-cli/**/*
-docker:
-  - docker/**/*
-go-client:
-  - go-client/**/*
-java-client:
-  - java-client/**/*
-nodejs-client:
-  - nodejs-client/**/*
-pegic:
-  - pegic/**/*
-python-client:
-  - python-client/**/*
-scala-client:
-  - scala-client/**/*
-thirdparty:
-  - thirdparty/**/*
-thrift:
-  - '**/*.thrift'
-docs:
-  - '**/*.md'
-scripts:
-  - 'scripts/**/*'
-  - '**/*.sh'
-build:
-  - 'cmake_modules/**/*'
-  - '**/CMakeLists.txt'
-  - 'run.sh'
-cpp:
-  # TODO(yingchun): add more fine-grained labels
-  - 'src/**/*.h'
-  - 'src/**/*.cpp'
+
+# the cluster that this collector is binding
+cluster_name : "onebox"
+
+# the meta server addresses of the cluster.
+meta_servers: 
+  - 127.0.0.1:34601
+  - 127.0.0.1:34602
+
+# local server port
+port : 34101
+
+metrics:
+  # use falcon as monitoring system.
+  sink : falcon 
+  report_interval : 10s
+
+prometheus:
+  # the exposed port for prometheus exposer
+  exposer_port : 1111 
+
+falcon_agent:
+  # the host IP of falcon agent
+  host : "127.0.0.1"
+  port : 1988
+  http_path : "/v1/push"
+
+available_detect:
+  table_name : test
diff --git a/collector/go.mod b/collector/go.mod
new file mode 100644
index 000000000..7288f318d
--- /dev/null
+++ b/collector/go.mod
@@ -0,0 +1,47 @@
+// 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.
+
+module github.com/pegasus-kv/collector
+
+go 1.13
+
+require (
+	github.com/ajg/form v1.5.1 // indirect
+	github.com/apache/incubator-pegasus/go-client v0.0.0-20220526071020-be5634371701
+	github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072 // indirect
+	github.com/google/go-querystring v1.0.0 // indirect
+	github.com/imkira/go-interpol v1.1.0 // indirect
+	github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 // indirect
+	github.com/kataras/iris/v12 v12.1.8
+	github.com/mattn/go-isatty v0.0.12 // indirect
+	github.com/moul/http2curl v1.0.0 // indirect
+	github.com/prometheus/client_golang v1.8.0
+	github.com/sergi/go-diff v1.1.0 // indirect
+	github.com/sirupsen/logrus v1.7.0
+	github.com/spf13/viper v1.7.1
+	github.com/stretchr/testify v1.6.1
+	github.com/tidwall/gjson v1.14.0
+	github.com/valyala/fasthttp v1.16.0 // indirect
+	github.com/xeipuuv/gojsonschema v1.2.0 // indirect
+	github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0 // indirect
+	github.com/yudai/gojsondiff v1.0.0 // indirect
+	github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 // indirect
+	github.com/yudai/pp v2.0.1+incompatible // indirect
+	gopkg.in/natefinch/lumberjack.v2 v2.0.0
+	gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637
+	k8s.io/apimachinery v0.0.0-20191123233150-4c4803ed55e3
+)
diff --git a/collector/go.sum b/collector/go.sum
new file mode 100644
index 000000000..6efe8418b
--- /dev/null
+++ b/collector/go.sum
@@ -0,0 +1,739 @@
+cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
+cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
+cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
+cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU=
+cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY=
+cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc=
+cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0=
+cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o=
+cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
+cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk=
+cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I=
+cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw=
+dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
+github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
+github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
+github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
+github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
+github.com/CloudyKit/fastprinter v0.0.0-20200109182630-33d98a066a53 h1:sR+/8Yb4slttB4vD+b9btVEnWgL3Q00OBTzVT8B9C0c=
+github.com/CloudyKit/fastprinter v0.0.0-20200109182630-33d98a066a53/go.mod h1:+3IMCy2vIlbG1XG/0ggNQv0SvxCAIpPM5b1nCz56Xno=
+github.com/CloudyKit/jet/v3 v3.0.0 h1:1PwO5w5VCtlUUl+KTOBsTGZlhjWkcybsGaAau52tOy8=
+github.com/CloudyKit/jet/v3 v3.0.0/go.mod h1:HKQPgSJmdK8hdoAbKUUWajkHyHo4RaU5rMdUywE7VMo=
+github.com/Joker/hpp v1.0.0 h1:65+iuJYdRXv/XyN62C1uEmmOx3432rNG/rKlX6V7Kkc=
+github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY=
+github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0=
+github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ=
+github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
+github.com/PuerkitoBio/purell v1.0.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
+github.com/PuerkitoBio/urlesc v0.0.0-20160726150825-5bd2802263f2/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
+github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398 h1:WDC6ySpJzbxGWFh4aMxFFC28wwGp5pEuoTtvA4q/qQ4=
+github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0=
+github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo=
+github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
+github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g=
+github.com/afex/hystrix-go v0.0.0-20180502004556-fa1af6a1f4f5/go.mod h1:SkGFH1ia65gfNATL8TAiHDNxPzPdmEL5uirI2Uyuz6c=
+github.com/agiledragon/gomonkey v2.0.2+incompatible h1:eXKi9/piiC3cjJD1658mEE2o3NjkJ5vDLgYjCQu0Xlw=
+github.com/agiledragon/gomonkey v2.0.2+incompatible/go.mod h1:2NGfXu1a80LLr2cmWXGBDaHEjb1idR6+FVlX5T3D9hw=
+github.com/ajg/form v1.5.1 h1:t9c7v8JUKu/XxOGBU0yjNpaMloxGEJhUkqFRq0ibGeU=
+github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY=
+github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
+github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
+github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
+github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
+github.com/andybalholm/brotli v1.0.0 h1:7UCwP93aiSfvWpapti8g88vVVGp2qqtGyePsSuDafo4=
+github.com/andybalholm/brotli v1.0.0/go.mod h1:loMXtMfwqflxFJPmdbJO0a3KNoPuLBgiu3qAvBg8x/Y=
+github.com/apache/incubator-pegasus/go-client v0.0.0-20220526071020-be5634371701 h1:J9d8jaC0rKlnf8iQuImyBiDrZnl794JlGMXX8DhMNrs=
+github.com/apache/incubator-pegasus/go-client v0.0.0-20220526071020-be5634371701/go.mod h1:vOnSzVwVYmatouq8qEBX+yWV4AvoCTLRpQn6JA+qojs=
+github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
+github.com/apache/thrift v0.13.0 h1:5hryIiq9gtn+MiLVn0wP37kb/uTeRZgN08WoCsAhIhI=
+github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
+github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
+github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8=
+github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY=
+github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
+github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A=
+github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU=
+github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
+github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g=
+github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible h1:Ppm0npCCsmuR9oQaBtRuZcmILVE74aXE+AmrJj8L2ns=
+github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g=
+github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
+github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
+github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
+github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
+github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
+github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84=
+github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ=
+github.com/cenkalti/backoff v2.2.1+incompatible h1:tNowT99t7UNflLxfYYSlKYsBpXdEet03Pg2g16Swow4=
+github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
+github.com/cenkalti/backoff/v4 v4.1.0 h1:c8LkOFQTzuO0WBM/ae5HdGQuZPfPxp7lqBRwQRm4fSc=
+github.com/cenkalti/backoff/v4 v4.1.0/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw=
+github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
+github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
+github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
+github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY=
+github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
+github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE=
+github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
+github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
+github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
+github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk=
+github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
+github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
+github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk=
+github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
+github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
+github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
+github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
+github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
+github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
+github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE=
+github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU=
+github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY=
+github.com/davecgh/go-spew v0.0.0-20151105211317-5215b55f46b2/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
+github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4=
+github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
+github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw=
+github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no=
+github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96/go.mod h1:Qh8CwZgvJUkLughtfhJv5dyTYa91l1fOUCrgjqmcifM=
+github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
+github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk=
+github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs=
+github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU=
+github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I=
+github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M=
+github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o=
+github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM=
+github.com/elazarl/goproxy v0.0.0-20170405201442-c4fc26588b6e/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc=
+github.com/emicklei/go-restful v0.0.0-20170410110728-ff4f55a20633/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs=
+github.com/envoyproxy/go-control-plane v0.6.9/go.mod h1:SBwIajubJHhxtWwsL9s8ss4safvEdbitLhGGK48rN6g=
+github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
+github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
+github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw=
+github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=
+github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072 h1:DddqAaWDpywytcG8w/qoQ5sAN8X12d3Z3koB0C3Rxsc=
+github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8=
+github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
+github.com/fatih/structs v1.1.0 h1:Q7juDM0QtcnhCpeyLGQKyg4TOIghuNXrkL32pHAUMxo=
+github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M=
+github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw=
+github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g=
+github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4=
+github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20=
+github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I=
+github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
+github.com/gavv/httpexpect v2.0.0+incompatible h1:1X9kcRshkSKEjNJJxX9Y9mQ5BRfbxU5kORdjhlA1yX8=
+github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc=
+github.com/ghodss/yaml v0.0.0-20150909031657-73d445a93680/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
+github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
+github.com/go-check/check v0.0.0-20180628173108-788fd7840127 h1:0gkP6mzaMqkmpcJYCFOLkIBwI7xFExG03bbkOkCvUPI=
+github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98=
+github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
+github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
+github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o=
+github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
+github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
+github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
+github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas=
+github.com/go-openapi/jsonpointer v0.0.0-20160704185906-46af16f9f7b1/go.mod h1:+35s3my2LFTysnkMfxsJBAMHj/DoqoB9knIWoYG/Vk0=
+github.com/go-openapi/jsonreference v0.0.0-20160704190145-13c6e3589ad9/go.mod h1:W3Z9FmVs9qj+KR4zFKmDPGiLdk1D9Rlm7cyMvf57TTg=
+github.com/go-openapi/spec v0.0.0-20160808142527-6aced65f8501/go.mod h1:J8+jY1nAiCcj+friV/PDoE1/3eeccG9LYBs0tYvLOWc=
+github.com/go-openapi/swag v0.0.0-20160704191624-1d0bd113de87/go.mod h1:DXUve3Dpr1UfpPtxFw+EFuQ41HhCWZfha5jSVRG7C7I=
+github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w=
+github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
+github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo=
+github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw=
+github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM=
+github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s=
+github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
+github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
+github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
+github.com/gogo/protobuf v1.2.2-0.20190723190241-65acae22fc9d/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
+github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
+github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
+github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
+github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
+github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y=
+github.com/golang/protobuf v0.0.0-20161109072736-4bd1920723d7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
+github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
+github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
+github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
+github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
+github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/protobuf v1.4.3 h1:JjCZWpVbqXDqFVmTfYWEVTMIYrL/NPdPSCHPJ0T/raM=
+github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
+github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
+github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4=
+github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
+github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
+github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
+github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
+github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
+github.com/google/go-cmp v0.4.0 h1:xsAVV57WRhGj6kEIi8ReJzQlHHqcBYCElAvkovg3B/4=
+github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/google/go-querystring v1.0.0 h1:Xkwi/a1rcvNg1PPYe5vI8GbeBY/jrVuDX5ASuANWTrk=
+github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
+github.com/google/gofuzz v0.0.0-20161122191042-44d81051d367/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
+github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
+github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
+github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
+github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
+github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
+github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
+github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
+github.com/googleapis/gnostic v0.0.0-20170729233727-0c5108395e2d/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY=
+github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8=
+github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
+github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg=
+github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs=
+github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs=
+github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ=
+github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
+github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc=
+github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
+github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
+github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
+github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
+github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
+github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
+github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q=
+github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE=
+github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
+github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
+github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
+github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
+github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
+github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
+github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk=
+github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU=
+github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU=
+github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4=
+github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
+github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
+github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA=
+github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90=
+github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
+github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
+github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4=
+github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ=
+github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64=
+github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
+github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
+github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
+github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
+github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
+github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg=
+github.com/imkira/go-interpol v1.1.0 h1:KIiKr0VSG2CUW1hl1jpiyuzuJeKUUpC8iM1AIE7N1Vk=
+github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA=
+github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
+github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo=
+github.com/iris-contrib/blackfriday v2.0.0+incompatible h1:o5sHQHHm0ToHUlAJSTjW9UWicjJSDDauOOQ2AHuIVp4=
+github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI=
+github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0=
+github.com/iris-contrib/jade v1.1.3 h1:p7J/50I0cjo0wq/VWVCDFd8taPJbuFC+bq23SniRFX0=
+github.com/iris-contrib/jade v1.1.3/go.mod h1:H/geBymxJhShH5kecoiOCSssPX7QWYH7UaeZTSWddIk=
+github.com/iris-contrib/pongo2 v0.0.1 h1:zGP7pW51oi5eQZMIlGA3I+FHY9/HOQWDB+572yin0to=
+github.com/iris-contrib/pongo2 v0.0.1/go.mod h1:Ssh+00+3GAZqSQb30AvBRNxBx7rf0GqwkjqxNd0u65g=
+github.com/iris-contrib/schema v0.0.1 h1:10g/WnoRR+U+XXHWKBHeNy/+tZmM2kcAVGLOsz+yaDA=
+github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw=
+github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
+github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
+github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
+github.com/json-iterator/go v0.0.0-20180612202835-f2b4162afba3/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
+github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
+github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/json-iterator/go v1.1.10 h1:Kz6Cvnvv2wGdaG/V8yMvfkmNiXq9Ya2KUv4rouJJr68=
+github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
+github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
+github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo=
+github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
+github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
+github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM=
+github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 h1:uC1QfSlInpQF+M0ao65imhwqKnz3Q2z/d8PWZRMQvDM=
+github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k=
+github.com/kataras/golog v0.0.10 h1:vRDRUmwacco/pmBAm8geLn8rHEdc+9Z4NAr5Sh7TG/4=
+github.com/kataras/golog v0.0.10/go.mod h1:yJ8YKCmyL+nWjERB90Qwn+bdyBZsaQwU3bTVFgkFIp8=
+github.com/kataras/iris/v12 v12.1.8 h1:O3gJasjm7ZxpxwTH8tApZsvf274scSGQAUpNe47c37U=
+github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYbq3UhfoFmE=
+github.com/kataras/neffos v0.0.14/go.mod h1:8lqADm8PnbeFfL7CLXh1WHw53dG27MC3pgi2R1rmoTE=
+github.com/kataras/pio v0.0.2 h1:6NAi+uPJ/Zuid6mrAKlgpbI11/zK/lV4B2rxWaJN98Y=
+github.com/kataras/pio v0.0.2/go.mod h1:hAoW0t9UmXi4R5Oyq5Z4irTbaTsOemSrDGUtaTl7Dro=
+github.com/kataras/sitemap v0.0.5 h1:4HCONX5RLgVy6G4RkYOV3vKNcma9p236LdGOipJsaFE=
+github.com/kataras/sitemap v0.0.5/go.mod h1:KY2eugMKiPwsJgx7+U103YZehfvNGOXURubcGyk0Bz8=
+github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
+github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
+github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
+github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A=
+github.com/klauspost/compress v1.10.7 h1:7rix8v8GpI3ZBb0nSozFRgbtXKv+hOe+qfEpZqybrAg=
+github.com/klauspost/compress v1.10.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs=
+github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
+github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
+github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
+github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
+github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
+github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
+github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM=
+github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4=
+github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ=
+github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
+github.com/magiconair/properties v1.8.1 h1:ZC2Vc7/ZFkGmsVC9KvOjumD+G5lXy2RtTKyzRKO2BQ4=
+github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
+github.com/mailru/easyjson v0.0.0-20160728113105-d5b7844b561a/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
+github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4=
+github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU=
+github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
+github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
+github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY=
+github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
+github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
+github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw=
+github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
+github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
+github.com/mediocregopher/radix/v3 v3.4.2/go.mod h1:8FL3F6UQRXHXIBSPUs5h0RybMF8i4n7wVopoX3x7Bv8=
+github.com/microcosm-cc/bluemonday v1.0.2 h1:5lPfLTTAvAbtS0VqT+94yOtFnGfUWYyx0+iToC3Os3s=
+github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc=
+github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
+github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
+github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
+github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
+github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
+github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg=
+github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY=
+github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
+github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE=
+github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
+github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
+github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
+github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
+github.com/modern-go/reflect2 v0.0.0-20180320133207-05fbef0ca5da/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI=
+github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/moul/http2curl v1.0.0 h1:dRMWoAtb+ePxMlLkrCbAqh4TlPHXvoGUSQ323/9Zahs=
+github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ=
+github.com/munnerz/goautoneg v0.0.0-20120707110453-a547fc61f48d/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ=
+github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
+github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
+github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f/go.mod h1:ZdcZmHo+o7JKHSa8/e818NopupXU1YMK5fe1lsApnBw=
+github.com/nats-io/jwt v0.3.0/go.mod h1:fRYCDE99xlTsqUzISS1Bi75UBJ6ljOJQOAAu5VglpSg=
+github.com/nats-io/jwt v0.3.2/go.mod h1:/euKqTS1ZD+zzjYrY7pseZrTtWQSjujC7xjPc8wL6eU=
+github.com/nats-io/nats-server/v2 v2.1.2/go.mod h1:Afk+wRZqkMQs/p45uXdrVLuab3gwv3Z8C4HTBu8GD/k=
+github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzEE/Zbp4w=
+github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
+github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w=
+github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c=
+github.com/oklog/oklog v0.3.2/go.mod h1:FCV+B7mhrz4o+ueLpx+KqkyXRGMWOYEvfiXtdGtbWGs=
+github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA=
+github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
+github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
+github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
+github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
+github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
+github.com/onsi/ginkgo v1.10.1 h1:q/mM8GF/n0shIN8SaAZ0V+jnLPzen6WIVZdiwrRlMlo=
+github.com/onsi/ginkgo v1.10.1/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
+github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
+github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
+github.com/onsi/gomega v1.7.0 h1:XPnZz8VVBHjVsy1vzJmRwIcSwiUO+JFfrv/xGiigmME=
+github.com/onsi/gomega v1.7.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
+github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk=
+github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis=
+github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74=
+github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
+github.com/openzipkin-contrib/zipkin-go-opentracing v0.4.5/go.mod h1:/wsWhb9smxSfWAKL3wpBW7V8scJMt8N8gnaMCS9E/cA=
+github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw=
+github.com/openzipkin/zipkin-go v0.2.1/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4=
+github.com/openzipkin/zipkin-go v0.2.2/go.mod h1:NaW6tEwdmWMaCDZzg8sh+IBNOxHMPnhQw8ySjnjRyN4=
+github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM=
+github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
+github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k=
+github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc=
+github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
+github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac=
+github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc=
+github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
+github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
+github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
+github.com/pmezard/go-difflib v0.0.0-20151028094244-d8ed2627bdf0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
+github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI=
+github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
+github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs=
+github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso=
+github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
+github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og=
+github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M=
+github.com/prometheus/client_golang v1.8.0 h1:zvJNkoCFAnYFNC24FV8nW4JdRJ3GIFcLbg65lL/JDcw=
+github.com/prometheus/client_golang v1.8.0/go.mod h1:O9VU6huf47PktckDQfMTX0Y8tY0/7TSWwj+ITvv0TnM=
+github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
+github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
+github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M=
+github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
+github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
+github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA=
+github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo=
+github.com/prometheus/common v0.14.0 h1:RHRyE8UocrbjU+6UvRzwi6HjiDfxrrBU91TtbKzkGp4=
+github.com/prometheus/common v0.14.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s=
+github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
+github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
+github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
+github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
+github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A=
+github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
+github.com/prometheus/procfs v0.2.0 h1:wH4vA7pcjKuZzjF7lM8awk4fnuJO6idemZXoKnULUx4=
+github.com/prometheus/procfs v0.2.0/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
+github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU=
+github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
+github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
+github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
+github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
+github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
+github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
+github.com/ryanuber/columnize v2.1.0+incompatible h1:j1Wcmh8OrK4Q7GXY+V7SVSY8nUWQxHW5TkBe7YUl+2s=
+github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
+github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
+github.com/schollz/closestmatch v2.1.0+incompatible h1:Uel2GXEpJqOWBrlyI+oY9LTiyyjYS17cCYRqP13/SHk=
+github.com/schollz/closestmatch v2.1.0+incompatible/go.mod h1:RtP1ddjLong6gTkbtmuhtR2uUrrJOpYzYRvbcPAid+g=
+github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc=
+github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0=
+github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM=
+github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo=
+github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc=
+github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
+github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
+github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88=
+github.com/sirupsen/logrus v1.7.0 h1:ShrD1U9pZB12TX0cVy0DtePoCH97K8EtX+mg7ZARUtM=
+github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0=
+github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM=
+github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
+github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s=
+github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
+github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
+github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY=
+github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
+github.com/spf13/afero v1.1.2 h1:m8/z1t7/fwjysjQRYbP0RD+bUIF/8tJwPdEZsI83ACI=
+github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ=
+github.com/spf13/cast v1.3.0 h1:oget//CVOEoFewqQxwr0Ej5yjygnqGkvggSE/gB35Q8=
+github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE=
+github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
+github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU=
+github.com/spf13/jwalterweatherman v1.0.0 h1:XHEdyB+EcvlqZamSM4ZOMGlc93t6AcsBEu9Gc1vn7yk=
+github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo=
+github.com/spf13/pflag v0.0.0-20170130214245-9ff6c6923cff/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
+github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA=
+github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
+github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s=
+github.com/spf13/viper v1.7.1 h1:pM5oEahlgWv/WnHXpgbKz7iLIxRf65tye2Ci+XFK5sk=
+github.com/spf13/viper v1.7.1/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg=
+github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw=
+github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw=
+github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI=
+github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/testify v0.0.0-20151208002404-e3a8ff8ce365/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
+github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
+github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
+github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
+github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0=
+github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
+github.com/subosito/gotenv v1.2.0 h1:Slr1R9HxAlEKefgq5jn9U+DnETlIUa6HfgEzj0g5d7s=
+github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw=
+github.com/tidwall/gjson v1.14.0 h1:6aeJ0bzojgWLa82gDQHcx3S0Lr/O51I9bJ5nv6JFx5w=
+github.com/tidwall/gjson v1.14.0/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk=
+github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA=
+github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM=
+github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs=
+github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
+github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
+github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0=
+github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
+github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
+github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6KllzawFIhcdPw=
+github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
+github.com/valyala/fasthttp v1.16.0 h1:9zAqOYLl8Tuy3E5R6ckzGDJ1g8+pw15oQp2iL9Jl6gQ=
+github.com/valyala/fasthttp v1.16.0/go.mod h1:YOKImeEosDdBPnxc0gy7INqi3m1zK6A+xl6TwOBhHCA=
+github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio=
+github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f h1:J9EGpcZtP0E/raorCMxlFGSTBrsSlaDGf3jU/qvAE2c=
+github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU=
+github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415 h1:EzJWgHovont7NscjpAxXsDA8S8BMYve8Y5+7cuRE7R0=
+github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ=
+github.com/xeipuuv/gojsonschema v1.2.0 h1:LhYJRs+L4fBtjZUfuSZIKGeVu0QRy8e5Xi7D17UxZ74=
+github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y=
+github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
+github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q=
+github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0 h1:6fRhSjgLCkTD3JnJxvaJ4Sj+TYblw757bqYgZaOq5ZY=
+github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI=
+github.com/yudai/gojsondiff v1.0.0 h1:27cbfqXLVEJ1o8I6v3y9lg8Ydm53EKqHXAOMxEGlCOA=
+github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg=
+github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 h1:BHyfKlQyqbsFN5p3IfnEUduWvb9is428/nNb5L3U01M=
+github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM=
+github.com/yudai/pp v2.0.1+incompatible h1:Q4//iY4pNF6yPLZIigmvcl7k/bPgrcTPIFIcmawg5bI=
+github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc=
+go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
+go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
+go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg=
+go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
+go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
+go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
+go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
+go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
+go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
+go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
+go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
+go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
+go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA=
+go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
+go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
+golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
+golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
+golang.org/x/crypto v0.0.0-20191227163750-53104e6ec876/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI=
+golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
+golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
+golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8=
+golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek=
+golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY=
+golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
+golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0=
+golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
+golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
+golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
+golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
+golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE=
+golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o=
+golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc=
+golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
+golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
+golang.org/x/net v0.0.0-20170114055629-f2499483f923/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190125091013-d26f9f9a57f3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
+golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
+golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
+golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20191004110552-13f9640d40b9/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20191105084925-a882066a44e0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
+golang.org/x/net v0.0.0-20200602114024-627f9648deb9/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
+golang.org/x/net v0.0.0-20200625001655-4c5254603344 h1:vGXIOMxbNfDTk/aXCmfdLgkrSV+Z2tcbze+pEc3v5W4=
+golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
+golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
+golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
+golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
+golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
+golang.org/x/sys v0.0.0-20170830134202-bb24a47a89ea/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
+golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/sys v0.0.0-20201015000850-e3ed0017c211 h1:9UQO31fZ+0aKQOFldThf7BKPMJTiBfWycGh/u3UoO88=
+golang.org/x/sys v0.0.0-20201015000850-e3ed0017c211/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
+golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
+golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs=
+golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
+golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
+golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20181011042414-1f849cf54d09/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
+golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
+golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
+golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
+golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
+golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
+golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
+golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
+golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
+golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
+golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
+golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4=
+golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk=
+google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE=
+google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M=
+google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
+google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
+google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI=
+google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
+google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
+google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
+google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
+google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0=
+google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
+google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
+google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
+google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
+google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
+google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s=
+google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
+google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
+google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8=
+google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
+google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
+google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
+google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM=
+google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
+google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
+google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
+google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
+google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
+google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
+google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
+google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
+google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
+google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
+google.golang.org/protobuf v1.23.0 h1:4MY060fB1DLGMB/7MBTLnwQUY6+F09GEiz6SsrNqyzM=
+google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
+gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo=
+gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw=
+gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
+gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
+gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
+gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o=
+gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
+gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
+gopkg.in/ini.v1 v1.51.1 h1:GyboHr4UqMiLUybYjd22ZjQIKEJEpgtLXtuGbR21Oho=
+gopkg.in/ini.v1 v1.51.1/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
+gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA=
+gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8=
+gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k=
+gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo=
+gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
+gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
+gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637 h1:yiW+nvdHb9LVqSHQBXfZCieqV4fzYhNBql77zY0ykqs=
+gopkg.in/tomb.v2 v2.0.0-20161208151619-d5d1b5820637/go.mod h1:BHsqpu/nsuzkT5BpiH1EMZPLyqSMM8JbIavyFACoFNk=
+gopkg.in/warnings.v0 v0.1.2/go.mod h1:jksf8JmL6Qr/oQM2OXTHunEvvTAsrWBLb6OOjuVWRNI=
+gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
+gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU=
+gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
+gopkg.in/yaml.v3 v3.0.0-20191120175047-4206685974f2/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
+gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo=
+gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
+honnef.co/go/tools v0.0.0-20180728063816-88497007e858/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
+honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg=
+k8s.io/apimachinery v0.0.0-20191123233150-4c4803ed55e3 h1:FErmbNIJruD5GT2oVEjtPn5Ar5+rcWJsC8/PPUkR0s4=
+k8s.io/apimachinery v0.0.0-20191123233150-4c4803ed55e3/go.mod h1:b9qmWdKlLuU9EBh+06BtLcSf/Mu89rWL33naRxs1uZg=
+k8s.io/gengo v0.0.0-20190128074634-0689ccc1d7d6/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0=
+k8s.io/klog v0.0.0-20181102134211-b9b56d5dfc92/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk=
+k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I=
+k8s.io/kube-openapi v0.0.0-20191107075043-30be4d16710a/go.mod h1:1TqjTSzOxsLGIKfj0lK8EeCP7K1iUG65v09OM0/WG5E=
+rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8=
+sigs.k8s.io/structured-merge-diff v0.0.0-20190525122527-15d366b2352e/go.mod h1:wWxsB5ozmmv/SG7nM11ayaAW51xMvak/t1r0CSlcokI=
+sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o=
+sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU=
diff --git a/collector/hotspot/algo.go b/collector/hotspot/algo.go
new file mode 100644
index 000000000..6b24419cf
--- /dev/null
+++ b/collector/hotspot/algo.go
@@ -0,0 +1,18 @@
+// 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.
+
+package hotspot
diff --git a/collector/main.go b/collector/main.go
new file mode 100644
index 000000000..d3d45d26c
--- /dev/null
+++ b/collector/main.go
@@ -0,0 +1,99 @@
+// 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.
+
+package main
+
+import (
+	"errors"
+	"fmt"
+	"os"
+	"os/signal"
+	"runtime"
+	"strings"
+	"syscall"
+
+	"github.com/pegasus-kv/collector/aggregate"
+	"github.com/pegasus-kv/collector/usage"
+	"github.com/pegasus-kv/collector/webui"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+	"gopkg.in/natefinch/lumberjack.v2"
+	"gopkg.in/tomb.v2"
+)
+
+// callerPrettifier simplifies the caller info
+func callerPrettifier(f *runtime.Frame) (function string, file string) {
+	function = f.Function[strings.LastIndex(f.Function, "/")+1:]
+	file = fmt.Sprint(f.File[strings.LastIndex(f.File, "/")+1:], ":", f.Line)
+	return function, file
+}
+
+// setupSignalHandler setup signal handler for collector
+func setupSignalHandler(shutdownFunc func()) {
+	closeSignalChan := make(chan os.Signal, 1)
+	signal.Notify(closeSignalChan,
+		syscall.SIGHUP,
+		syscall.SIGINT,
+		syscall.SIGTERM,
+		syscall.SIGQUIT)
+
+	go func() {
+		sig := <-closeSignalChan
+		log.Infof("got signal %s to exit", sig.String())
+		shutdownFunc()
+	}()
+}
+
+func main() {
+	// initialize logging
+	log.SetFormatter(&log.TextFormatter{
+		DisableColors:    true,
+		FullTimestamp:    true,
+		CallerPrettyfier: callerPrettifier,
+	})
+	log.SetOutput(&lumberjack.Logger{ // rolling log
+		Filename:  "./pegasus.log",
+		MaxSize:   50, // MegaBytes
+		MaxAge:    2,  // days
+		LocalTime: true,
+	})
+	log.SetReportCaller(true)
+
+	// TODO(wutao1): use args[1] as config path
+	viper.SetConfigFile("config.yml")
+	viper.SetConfigType("yaml")
+	if err := viper.ReadInConfig(); err != nil {
+		log.Fatal("failed to read config: ", err)
+		return
+	}
+
+	webui.StartWebServer()
+
+	tom := &tomb.Tomb{}
+	setupSignalHandler(func() {
+		tom.Kill(errors.New("collector terminates")) // kill other goroutines
+	})
+	tom.Go(func() error {
+		aggregate.Start(tom)
+		return nil
+	})
+	tom.Go(func() error {
+		usage.NewTableUsageRecorder().Start(tom)
+		return nil
+	})
+	<-tom.Dead() // gracefully wait until all goroutines dead
+}
diff --git a/collector/metrics/falcon_sink.go b/collector/metrics/falcon_sink.go
new file mode 100644
index 000000000..4ccaedfcb
--- /dev/null
+++ b/collector/metrics/falcon_sink.go
@@ -0,0 +1,144 @@
+// 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.
+
+package metrics
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"net/http"
+	"time"
+
+	"github.com/pegasus-kv/collector/aggregate"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+)
+
+type falconConfig struct {
+	falconAgentHost     string
+	falconAgentPort     uint32
+	falconAgentHTTPPath string
+
+	clusterName           string
+	port                  uint32
+	metricsReportInterval time.Duration
+}
+
+type falconSink struct {
+	cfg *falconConfig
+}
+
+type falconMetricData struct {
+	Endpoint    string  // the cluster name
+	Metric      string  // metric name
+	Timestamp   int64   // the reporting time in unix seconds
+	Step        int32   // the reporting time interval in seconds
+	Value       float64 // metric value
+	CounterType string  // GAUGE or COUNTER
+	Tags        string
+}
+
+func newFalconSink() *falconSink {
+	sink := &falconSink{}
+	sink.cfg = &falconConfig{
+		falconAgentHost:       viper.GetString("falcon_agent.host"),
+		falconAgentPort:       viper.GetUint32("falcon_agent.port"),
+		falconAgentHTTPPath:   viper.GetString("falcon_agent.http_path"),
+		clusterName:           viper.GetString("cluster_name"),
+		port:                  viper.GetUint32("port"),
+		metricsReportInterval: viper.GetDuration("metrics.report_interval"),
+	}
+	return sink
+}
+
+func (m *falconMetricData) setData(name string, value float64, tags map[string]string) {
+	m.Metric = name
+	m.Value = value
+	m.Timestamp = time.Now().Unix()
+
+	firstTag := true
+	for k, v := range tags {
+		if firstTag {
+			firstTag = false
+		} else {
+			m.Tags += ","
+		}
+		m.Tags += k + "=" + v
+	}
+}
+
+func (m *falconMetricData) toJSON() []byte {
+	result, err := json.Marshal(m)
+	if err != nil {
+		log.Fatal("failed to marshall falcon metric to json: ", err)
+	}
+	return result
+}
+
+type falconDataSerializer struct {
+	buf *bytes.Buffer
+
+	mdata *falconMetricData
+}
+
+func (s *falconDataSerializer) serialize(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+	s.buf.WriteString("[")
+	for _, table := range stats {
+		for name, value := range table.Stats {
+			s.mdata.setData(name, value, map[string]string{
+				"entity": "table",
+				"table":  name,
+			})
+		}
+		s.buf.Write(s.mdata.toJSON())
+	}
+	for name, value := range allStats.Stats {
+		s.mdata.setData(name, value, map[string]string{
+			"entity": "cluster",
+		})
+		s.buf.Write(s.mdata.toJSON())
+	}
+}
+
+func (sink *falconSink) Report(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+	serializer := &falconDataSerializer{
+		buf: bytes.NewBuffer(nil),
+	}
+	serializer.mdata = &falconMetricData{
+		Endpoint:    sink.cfg.clusterName,
+		Step:        int32(sink.cfg.metricsReportInterval.Seconds()),
+		CounterType: "GAUGE",
+	}
+	serializer.serialize(stats, allStats)
+	sink.postHTTPData(serializer.buf.Bytes())
+}
+
+func (sink *falconSink) postHTTPData(data []byte) {
+	url := fmt.Sprintf("http://%s:%d/%s", sink.cfg.falconAgentHost, sink.cfg.falconAgentPort, sink.cfg.falconAgentHTTPPath)
+	resp, err := http.Post(url, "application/x-www-form-urlencoded", bytes.NewReader([]byte(data)))
+	if err == nil && resp.StatusCode != http.StatusOK {
+		err = errors.New(resp.Status)
+	}
+	if err != nil {
+		log.Errorf("failed to post metrics to falcon agent: %s", err)
+		return
+	}
+
+	defer resp.Body.Close()
+}
diff --git a/collector/metrics/prometheus_sink.go b/collector/metrics/prometheus_sink.go
new file mode 100644
index 000000000..e37fbc33d
--- /dev/null
+++ b/collector/metrics/prometheus_sink.go
@@ -0,0 +1,110 @@
+// 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.
+
+package metrics
+
+import (
+	"sync"
+
+	"github.com/pegasus-kv/collector/aggregate"
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+type prometheusMetricFamily struct {
+	metrics map[string]prometheus.Gauge
+}
+
+func (f *prometheusMetricFamily) set(name string, value float64) {
+	f.metrics[name].Set(value)
+}
+
+type prometheusSink struct {
+	tableMap map[int]*prometheusMetricFamily
+
+	tableLock sync.RWMutex
+
+	clusterMetric *prometheusMetricFamily
+
+	allTrackedMetrics []string
+}
+
+func newPrometheusSink() *prometheusSink {
+	sink := &prometheusSink{
+		tableMap:          make(map[int]*prometheusMetricFamily),
+		allTrackedMetrics: aggregate.AllMetrics(),
+	}
+	sink.clusterMetric = sink.newClusterMetricFamily()
+
+	aggregate.AddHookAfterTableDropped(func(appID int) {
+		// remove the metrics family belongs to the table
+		sink.tableLock.Lock()
+		for _, gauge := range sink.tableMap[appID].metrics {
+			prometheus.Unregister(gauge)
+		}
+		delete(sink.tableMap, appID)
+		sink.tableLock.Unlock()
+	})
+	return sink
+}
+
+func (sink *prometheusSink) Report(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+	for _, table := range stats {
+		sink.tableLock.Lock()
+		defer sink.tableLock.Unlock()
+
+		var mfamily *prometheusMetricFamily
+		var found bool
+		if mfamily, found = sink.tableMap[table.AppID]; !found {
+			mfamily = sink.newTableMetricFamily(table.TableName)
+			// insert table metrics family
+			sink.tableMap[table.AppID] = mfamily
+		}
+		fillStatsIntoGauges(table.Stats, mfamily)
+	}
+	fillStatsIntoGauges(allStats.Stats, sink.clusterMetric)
+}
+
+func fillStatsIntoGauges(stats map[string]float64, family *prometheusMetricFamily) {
+	for name, value := range stats {
+		family.set(name, value)
+	}
+}
+
+func (sink *prometheusSink) newTableMetricFamily(tableName string) *prometheusMetricFamily {
+	return sink.newMetricFamily(map[string]string{"table": tableName, "entity": "table"})
+}
+
+func (sink *prometheusSink) newClusterMetricFamily() *prometheusMetricFamily {
+	return sink.newMetricFamily(map[string]string{"entity": "cluster"})
+}
+
+func (sink *prometheusSink) newMetricFamily(labels map[string]string) *prometheusMetricFamily {
+	mfamily := &prometheusMetricFamily{
+		metrics: make(map[string]prometheus.Gauge),
+	}
+	for _, m := range sink.allTrackedMetrics {
+		// create and register a gauge
+		opts := prometheus.GaugeOpts{
+			Name:        m,
+			ConstLabels: labels,
+		}
+		gauge := prometheus.NewGauge(opts)
+		prometheus.MustRegister(gauge)
+		mfamily.metrics[m] = gauge
+	}
+	return mfamily
+}
diff --git a/collector/metrics/sink.go b/collector/metrics/sink.go
new file mode 100644
index 000000000..fbe590e86
--- /dev/null
+++ b/collector/metrics/sink.go
@@ -0,0 +1,53 @@
+// 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.
+
+package metrics
+
+import (
+	"github.com/pegasus-kv/collector/aggregate"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+)
+
+// Sink is the destination where the metrics are reported to.
+type Sink interface {
+
+	// Report the snapshot of metrics to the monitoring system. The report can possibly fail.
+	Report(stats []aggregate.TableStats, allStats aggregate.ClusterStats)
+}
+
+// NewSink creates a Sink which reports metrics to the configured monitoring system.
+func NewSink() Sink {
+	var sink Sink
+	cfgSink := viper.Get("metrics.sink")
+	if cfgSink == "falcon" {
+		sink = newFalconSink()
+	} else if cfgSink == "prometheus" {
+		sink = newPrometheusSink()
+	} else {
+		log.Fatalf("invalid metrics_sink = %s", cfgSink)
+		return nil
+	}
+
+	aggregate.AddHookAfterTableStatEmitted(func(stats []aggregate.TableStats, allStats aggregate.ClusterStats) {
+		go func() {
+			sink.Report(stats, allStats)
+		}()
+	})
+
+	return sink
+}
diff --git a/collector/templates/index.html b/collector/templates/index.html
new file mode 100644
index 000000000..fc3de7aa1
--- /dev/null
+++ b/collector/templates/index.html
@@ -0,0 +1,80 @@
+<!--
+  ~ 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.
+-->
+
+<!DOCTYPE html>
+<html>
+
+<head>
+    <meta charset="utf-8">
+    <!--responsive support-->
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <title>Pegasus Collector</title>
+    <link rel="stylesheet" href="https://cdn.jsdelivr.net/npm/bulma@0.9.0/css/bulma.min.css">
+</head>
+
+<nav class="navbar is-light" style="padding-left: 2rem;">
+    <div class="navbar-brand">
+        <a class="navbar-item" href="/"><strong>Pegasus Collector</strong></a>
+    </div>
+</nav>
+<main>
+    <div class="container" style="padding: 2rem;">
+        <h1 class="title" style="padding-bottom: 1rem;">Cluster Stats</h1>
+        <table class="table is-bordered is-striped is-fullwidth">
+            <thead>
+                <tr>
+                    <th>Perf-Counter</th>
+                    {{range .PerfIDs}}
+                    <th>{{.}}</th>
+                    {{end}}
+                </tr>
+            </thead>
+            <tbody>
+                {{range .PerfCounters}}
+                <tr>
+                    <td>{{.PerfCounter}}</td>
+                    {{range .Values}}
+                    <td>{{.}}</td>
+                    {{end}}
+                </tr>
+                {{end}}
+            </tbody>
+        </table>
+    </div>
+    <div class="container" style="padding: 2rem;">
+        <h1 class="title" style="padding-bottom: 1rem;">Tables</h1>
+        <table class="table is-bordered is-striped is-fullwidth">
+            <thead>
+                <tr>
+                    <th>Table Name</th>
+                    <th>Link</th>
+                </tr>
+            </thead>
+            <tbody>
+                {{range .Tables}}
+                <tr>
+                    <td>{{.TableName}}</td>
+                </tr>
+                {{end}}
+            </tbody>
+        </table>
+    </div>
+</main>
+
+</html>
\ No newline at end of file
diff --git a/collector/usage/usage_recorder.go b/collector/usage/usage_recorder.go
new file mode 100644
index 000000000..6a6784d39
--- /dev/null
+++ b/collector/usage/usage_recorder.go
@@ -0,0 +1,115 @@
+// 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.
+
+package usage
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	"github.com/apache/incubator-pegasus/go-client/pegasus"
+	"github.com/pegasus-kv/collector/aggregate"
+	log "github.com/sirupsen/logrus"
+	"github.com/spf13/viper"
+	"gopkg.in/tomb.v2"
+)
+
+// TableUsageRecorder records the usage of each table into a Pegasus table.
+// The usage statistics can be used for service cost calculation.
+type TableUsageRecorder interface {
+
+	// Start recording until the ctx cancelled. This method will block the current thread.
+	Start(tom *tomb.Tomb)
+}
+
+// NewTableUsageRecorder returns an instance of TableUsageRecorder
+func NewTableUsageRecorder() TableUsageRecorder {
+	return &tableUsageRecorder{
+		usageStatApp: viper.GetString("usage_stat_app"),
+	}
+}
+
+type tableUsageRecorder struct {
+	client pegasus.Client
+	table  pegasus.TableConnector
+
+	usageStatApp string
+}
+
+func (rec *tableUsageRecorder) Start(tom *tomb.Tomb) {
+	if rec.usageStatApp == "" {
+		// if no stat app is specified, usage recorder is considered as disabled.
+		return
+	}
+
+	metaServer := viper.GetString("meta_server")
+	rec.client = pegasus.NewClient(pegasus.Config{MetaServers: []string{metaServer}})
+	for {
+		var err error
+		rec.table, err = rec.client.OpenTable(tom.Context(context.TODO()), rec.usageStatApp)
+		if err != nil {
+			// retry indefinitely
+			log.Errorf("failed to open table: %s", err.Error())
+			sleepWait(tom, 15*time.Second)
+			continue
+		}
+		break
+	}
+
+	aggregate.AddHookAfterTableStatEmitted(func(stats []aggregate.TableStats, allStat aggregate.ClusterStats) {
+		rootCtx := tom.Context(context.TODO())
+		for _, s := range stats {
+			rec.writeTableUsage(rootCtx, &s)
+		}
+	})
+}
+
+func sleepWait(tom *tomb.Tomb, waitTime time.Duration) {
+	ticker := time.NewTicker(waitTime)
+	select {
+	case <-tom.Dying():
+		return
+	case <-ticker.C:
+	}
+}
+
+func (rec *tableUsageRecorder) writeTableUsage(ctx context.Context, tb *aggregate.TableStats) {
+	hashKey := []byte(fmt.Sprintf("%d", tb.Timestamp.Unix()))
+	sortkey := []byte("cu")
+
+	readCU := tb.Stats["recent_read_cu"]
+	writeCU := tb.Stats["recent_write_cu"]
+	value := []byte(fmt.Sprintf("{\"%d\":[%f, %f]}", tb.AppID, readCU, writeCU))
+
+	go func() {
+		maxRetryCount := 10
+		for maxRetryCount > 0 {
+			// TODO(wutao): set rpc timeout
+			err := rec.table.Set(ctx, hashKey, sortkey, value)
+			if err == nil {
+				break
+			}
+			log.Errorf("failed to write cu [timestamp: %s, appid: %d, readcu: %f, writecu: %f]",
+				tb.Timestamp.Local().String(),
+				tb.AppID,
+				readCU,
+				writeCU)
+			maxRetryCount--
+		}
+	}()
+}
diff --git a/collector/webui/index.go b/collector/webui/index.go
new file mode 100644
index 000000000..0c0d26ff4
--- /dev/null
+++ b/collector/webui/index.go
@@ -0,0 +1,84 @@
+// 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.
+
+package webui
+
+import (
+	"github.com/kataras/iris/v12"
+	"github.com/pegasus-kv/collector/aggregate"
+)
+
+var indexPageClusterStats = []string{
+	"write_bytes",
+	"read_bytes",
+	"write_qps",
+	"read_qps",
+}
+
+func renderIndexClusterCharts(ctx iris.Context) {
+	type perfCounterHTML struct {
+		PerfCounter string
+		Values      []float64
+	}
+	var PerfCounters []*perfCounterHTML
+
+	snapshots := aggregate.SnapshotClusterStats()
+	for _, s := range indexPageClusterStats {
+		PerfCounters = append(PerfCounters, &perfCounterHTML{
+			PerfCounter: s,
+		})
+		p := PerfCounters[len(PerfCounters)-1]
+		for _, snapshot := range snapshots {
+			if v, found := snapshot.Stats[s]; found {
+				p.Values = append(p.Values, v)
+			}
+		}
+	}
+	ctx.ViewData("PerfCounters", PerfCounters)
+
+	var PerfIDs []string
+	for _, sn := range snapshots {
+		PerfIDs = append(PerfIDs, sn.Timestamp.Format("15:04:00"))
+	}
+	ctx.ViewData("PerfIDs", PerfIDs)
+}
+
+func indexHandler(ctx iris.Context) {
+	renderIndexClusterCharts(ctx)
+
+	// metaClient := client(viper.GetString("meta_server"))
+	// tables, err := metaClient.ListTables()
+	// if err != nil {
+	// 	ctx.ResponseWriter().WriteString("Failed to list tables from MetaServer")
+	// 	ctx.StatusCode(iris.StatusInternalServerError)
+	// 	return
+	// }
+	// type tableHTMLRow struct {
+	// 	TableName string
+	// 	Link      string
+	// }
+	// var Tables []tableHTMLRow
+	// for _, tb := range tables {
+	// 	Tables = append(Tables, tableHTMLRow{TableName: tb.TableName})
+	// }
+	// ctx.ViewData("Tables", Tables)
+
+	err := ctx.View("index.html")
+	if err != nil {
+		return
+	}
+}
diff --git a/collector/webui/tables.go b/collector/webui/tables.go
new file mode 100644
index 000000000..04bb5b10e
--- /dev/null
+++ b/collector/webui/tables.go
@@ -0,0 +1,47 @@
+// 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.
+
+package webui
+
+import (
+	"github.com/kataras/iris/v12"
+)
+
+func tablesHandler(ctx iris.Context) {
+	// metaClient := client.NewMetaClient(viper.GetString("meta_server"))
+	// tables, err := metaClient.ListTables()
+	// if err != nil {
+	// 	ctx.ResponseWriter().WriteString("Failed to list tables from MetaServer")
+	// 	ctx.StatusCode(iris.StatusInternalServerError)
+	// 	return
+	// }
+
+	// type tableHTMLRow struct {
+	// 	TableName string
+	// 	Link      string
+	// }
+	// var Tables []tableHTMLRow
+	// for _, tb := range tables {
+	// 	Tables = append(Tables, tableHTMLRow{TableName: tb.TableName})
+	// }
+	// ctx.ViewData("Tables", Tables)
+
+	err := ctx.View("tables.html")
+	if err != nil {
+		return
+	}
+}
diff --git a/collector/webui/webserver.go b/collector/webui/webserver.go
new file mode 100644
index 000000000..8c13dda9c
--- /dev/null
+++ b/collector/webui/webserver.go
@@ -0,0 +1,61 @@
+// 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.
+
+package webui
+
+import (
+	"context"
+	"time"
+
+	"github.com/kataras/iris/v12"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
+)
+
+// StartWebServer starts an iris-powered HTTP server.
+func StartWebServer() {
+	app := iris.New()
+	app.Get("/", indexHandler)
+	app.Get("/tables", tablesHandler)
+	app.Get("/metrics", func(ctx iris.Context) {
+		handler := promhttp.Handler()
+		handler.ServeHTTP(ctx.ResponseWriter(), ctx.Request())
+	})
+
+	iris.RegisterOnInterrupt(func() {
+		// gracefully shutdown on interrupt
+		timeout := 5 * time.Second
+		ctx, cancel := context.WithTimeout(context.Background(), timeout)
+		defer cancel()
+		err := app.Shutdown(ctx)
+		if err != nil {
+			return
+		}
+	})
+
+	// Register the view engine to the views,
+	// this will load the templates.
+	tmpl := iris.HTML("./templates", ".html")
+	tmpl.Reload(true)
+	app.RegisterView(tmpl)
+
+	go func() {
+		err := app.Listen(":8080")
+		if err != nil {
+			return
+		}
+	}()
+}


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


[incubator-pegasus] 14/23: feat(new_metrics): add partition-level metric entity and migrate partition-level metrics for greedy_load_balancer of meta (#1435)

Posted by wa...@apache.org.
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 69ca0ec97ce70ded9ebbffeff36b43c4d5efcaa7
Author: Dan Wang <wa...@apache.org>
AuthorDate: Fri Apr 14 13:37:13 2023 +0800

    feat(new_metrics): add partition-level metric entity and migrate partition-level metrics for greedy_load_balancer of meta (#1435)
    
    https://github.com/apache/incubator-pegasus/issues/1331
    
    In perf counters, all metrics of greedy_load_balancer are server-level, for
    example, the number of each kind of operations by greedy balancer, including
    moving primaries, copying primaries and copying secondaries.
    
    For new metrics, it is hoped that they are fine-grained, since sometimes we
    want to know which primaries are moved. Also, it is convenient to calculate
    table-level or server-level metrics by just aggregate on partition-level ones.
    
    The metrics of greedy_load_balancer that are changed to partition-level and
    migrated to new framework include: the number of balance operations by
    greedy balancer that are recently needed to be executed, move primaries,
    copy primaries, and copy secondaries.
    
    In addition to the metrics of greedy_load_balancer, we also change some
    metrics of server_state again to partition-level which have been migrated
    to table-level in https://github.com/apache/incubator-pegasus/pull/1431,
    for the reason that partition-level is considered more appropriate for them
    than table-level.  The metrics changed to partition-level include the number
    of times the configuration has been changed and the number of times the
    status of partition has been changed to unwritable or writable for a partition.
    
    To implement table-level metrics, partition-level metric entity is also added.
---
 src/common/fs_manager.cpp                   |   4 +-
 src/common/fs_manager.h                     |   4 +-
 src/meta/greedy_load_balancer.cpp           |  48 ++----
 src/meta/greedy_load_balancer.h             |   7 -
 src/meta/meta_split_service.cpp             |   5 +
 src/meta/server_state.cpp                   |  38 ++---
 src/meta/server_state.h                     |   2 +
 src/meta/server_state_restore.cpp           |   2 +-
 src/meta/table_metrics.cpp                  | 241 +++++++++++++++++++++++++---
 src/meta/table_metrics.h                    | 200 ++++++++++++++++++-----
 src/meta/test/backup_test.cpp               |   1 +
 src/meta/test/meta_split_service_test.cpp   |   4 +
 src/meta/test/update_configuration_test.cpp |   2 +-
 src/replica/test/replica_test.cpp           |   1 +
 src/utils/metrics.h                         |  20 ++-
 15 files changed, 443 insertions(+), 136 deletions(-)

diff --git a/src/common/fs_manager.cpp b/src/common/fs_manager.cpp
index 61ca99411..0541abca9 100644
--- a/src/common/fs_manager.cpp
+++ b/src/common/fs_manager.cpp
@@ -148,8 +148,8 @@ bool dir_node::update_disk_stat(const bool update_disk_status)
     disk_available_ratio = static_cast<int>(
         disk_capacity_mb == 0 ? 0 : std::round(disk_available_mb * 100.0 / disk_capacity_mb));
 
-    METRIC_CALL_SET_METHOD(disk_capacity, total_disk_capacity_mb, disk_capacity_mb);
-    METRIC_CALL_SET_METHOD(disk_capacity, avail_disk_capacity_mb, disk_available_mb);
+    METRIC_SET(disk_capacity, total_disk_capacity_mb, disk_capacity_mb);
+    METRIC_SET(disk_capacity, avail_disk_capacity_mb, disk_available_mb);
 
     if (!update_disk_status) {
         LOG_INFO("update disk space succeed: dir = {}, capacity_mb = {}, available_mb = {}, "
diff --git a/src/common/fs_manager.h b/src/common/fs_manager.h
index 6efe1b1bd..92fa1b048 100644
--- a/src/common/fs_manager.h
+++ b/src/common/fs_manager.h
@@ -51,8 +51,8 @@ public:
 
     const metric_entity_ptr &disk_metric_entity() const;
 
-    METRIC_DEFINE_SET_METHOD(total_disk_capacity_mb, int64_t)
-    METRIC_DEFINE_SET_METHOD(avail_disk_capacity_mb, int64_t)
+    METRIC_DEFINE_SET(total_disk_capacity_mb, int64_t)
+    METRIC_DEFINE_SET(avail_disk_capacity_mb, int64_t)
 
 private:
     const metric_entity_ptr _disk_metric_entity;
diff --git a/src/meta/greedy_load_balancer.cpp b/src/meta/greedy_load_balancer.cpp
index f8206b820..dc8ef4f9d 100644
--- a/src/meta/greedy_load_balancer.cpp
+++ b/src/meta/greedy_load_balancer.cpp
@@ -36,21 +36,23 @@
 #include "cluster_balance_policy.h"
 #include "greedy_load_balancer.h"
 #include "meta/load_balance_policy.h"
+#include "meta/meta_service.h"
 #include "meta/server_load_balancer.h"
+#include "meta/server_state.h"
+#include "meta/table_metrics.h"
 #include "meta_admin_types.h"
 #include "meta_data.h"
-#include "perf_counter/perf_counter.h"
 #include "runtime/rpc/rpc_address.h"
 #include "utils/command_manager.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
 #include "utils/math.h"
+#include "utils/metrics.h"
 
 namespace dsn {
 class gpid;
 
 namespace replication {
-class meta_service;
 
 DSN_DEFINE_bool(meta_server, balance_cluster, false, "whether to enable cluster balancer");
 DSN_TAG_VARIABLE(balance_cluster, FT_MUTABLE);
@@ -63,27 +65,6 @@ greedy_load_balancer::greedy_load_balancer(meta_service *_svc) : server_load_bal
     _cluster_balance_policy = std::make_unique<cluster_balance_policy>(_svc);
 
     ::memset(t_operation_counters, 0, sizeof(t_operation_counters));
-
-    // init perf counters
-    _balance_operation_count.init_app_counter("eon.greedy_balancer",
-                                              "balance_operation_count",
-                                              COUNTER_TYPE_NUMBER,
-                                              "balance operation count to be done");
-    _recent_balance_move_primary_count.init_app_counter(
-        "eon.greedy_balancer",
-        "recent_balance_move_primary_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "move primary count by balancer in the recent period");
-    _recent_balance_copy_primary_count.init_app_counter(
-        "eon.greedy_balancer",
-        "recent_balance_copy_primary_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "copy primary count by balancer in the recent period");
-    _recent_balance_copy_secondary_count.init_app_counter(
-        "eon.greedy_balancer",
-        "recent_balance_copy_secondary_count",
-        COUNTER_TYPE_VOLATILE_NUMBER,
-        "copy secondary count by balancer in the recent period");
 }
 
 greedy_load_balancer::~greedy_load_balancer() {}
@@ -228,34 +209,37 @@ bool greedy_load_balancer::check(meta_view view, migration_list &list)
 void greedy_load_balancer::report(const dsn::replication::migration_list &list,
                                   bool balance_checker)
 {
-    int counters[MAX_COUNT];
-    ::memset(counters, 0, sizeof(counters));
+#define __METRIC_INCREMENT(name)                                                                   \
+    METRIC_INCREMENT(balance_stats, name, action.first, balance_checker)
+
+    int counters[MAX_COUNT] = {0};
+    greedy_balance_stats balance_stats;
 
     counters[ALL_COUNT] = list.size();
     for (const auto &action : list) {
         switch (action.second.get()->balance_type) {
         case balancer_request_type::move_primary:
             counters[MOVE_PRI_COUNT]++;
+            __METRIC_INCREMENT(greedy_move_primary_operations);
             break;
         case balancer_request_type::copy_primary:
             counters[COPY_PRI_COUNT]++;
+            __METRIC_INCREMENT(greedy_copy_primary_operations);
             break;
         case balancer_request_type::copy_secondary:
             counters[COPY_SEC_COUNT]++;
+            __METRIC_INCREMENT(greedy_copy_secondary_operations);
             break;
         default:
             CHECK(false, "");
         }
     }
+
     ::memcpy(t_operation_counters, counters, sizeof(counters));
+    METRIC_SET_GREEDY_BALANCE_STATS(_svc->get_server_state()->get_table_metric_entities(),
+                                    balance_stats);
 
-    // update perf counters
-    _balance_operation_count->set(list.size());
-    if (!balance_checker) {
-        _recent_balance_move_primary_count->add(counters[MOVE_PRI_COUNT]);
-        _recent_balance_copy_primary_count->add(counters[COPY_PRI_COUNT]);
-        _recent_balance_copy_secondary_count->add(counters[COPY_SEC_COUNT]);
-    }
+#undef __METRIC_INCREMENT
 }
 } // namespace replication
 } // namespace dsn
diff --git a/src/meta/greedy_load_balancer.h b/src/meta/greedy_load_balancer.h
index 862a29c19..2284b7a61 100644
--- a/src/meta/greedy_load_balancer.h
+++ b/src/meta/greedy_load_balancer.h
@@ -40,7 +40,6 @@
 
 #include "meta/meta_data.h"
 #include "meta_admin_types.h"
-#include "perf_counter/perf_counter_wrapper.h"
 #include "server_load_balancer.h"
 
 namespace dsn {
@@ -86,12 +85,6 @@ private:
 
     std::unique_ptr<command_deregister> _get_balance_operation_count;
 
-    // perf counters
-    perf_counter_wrapper _balance_operation_count;
-    perf_counter_wrapper _recent_balance_move_primary_count;
-    perf_counter_wrapper _recent_balance_copy_primary_count;
-    perf_counter_wrapper _recent_balance_copy_secondary_count;
-
 private:
     void greedy_balancer(bool balance_checker);
     bool all_replica_infos_collected(const node_state &ns);
diff --git a/src/meta/meta_split_service.cpp b/src/meta/meta_split_service.cpp
index c95cf7a6e..33610fc6a 100644
--- a/src/meta/meta_split_service.cpp
+++ b/src/meta/meta_split_service.cpp
@@ -35,6 +35,7 @@
 #include "meta/meta_service.h"
 #include "meta/meta_state_service.h"
 #include "meta/server_state.h"
+#include "meta/table_metrics.h"
 #include "meta_admin_types.h"
 #include "meta_split_service.h"
 #include "meta_state_service_utils.h"
@@ -118,6 +119,7 @@ void meta_split_service::do_start_partition_split(std::shared_ptr<app_state> app
         app->partition_count *= 2;
         app->helpers->contexts.resize(app->partition_count);
         app->partitions.resize(app->partition_count);
+        _state->get_table_metric_entities().resize_partitions(app->app_id, app->partition_count);
         app->envs[replica_envs::SPLIT_VALIDATE_PARTITION_HASH] = "true";
 
         for (int i = 0; i < app->partition_count; ++i) {
@@ -553,10 +555,13 @@ void meta_split_service::do_cancel_partition_split(std::shared_ptr<app_state> ap
         LOG_INFO("app({}) update partition count on remote storage, new partition count is {}",
                  app->app_name,
                  app->partition_count / 2);
+
         zauto_write_lock l(app_lock());
+
         app->partition_count /= 2;
         app->helpers->contexts.resize(app->partition_count);
         app->partitions.resize(app->partition_count);
+        _state->get_table_metric_entities().resize_partitions(app->app_id, app->partition_count);
     };
 
     auto copy = *app;
diff --git a/src/meta/server_state.cpp b/src/meta/server_state.cpp
index 71fe7b1af..3edb04580 100644
--- a/src/meta/server_state.cpp
+++ b/src/meta/server_state.cpp
@@ -85,6 +85,7 @@
 #include "utils/config_api.h"
 #include "utils/flags.h"
 #include "utils/fmt_logging.h"
+#include "utils/metrics.h"
 #include "utils/string_conv.h"
 #include "utils/strings.h"
 
@@ -498,7 +499,7 @@ error_code server_state::sync_apps_to_remote_storage()
                   "invalid app name, name = {}",
                   kv_pair.second->app_name);
             _exist_apps.emplace(kv_pair.second->app_name, kv_pair.second);
-            _table_metric_entities.create_entity(kv_pair.first);
+            _table_metric_entities.create_entity(kv_pair.first, kv_pair.second->partition_count);
         }
     }
 
@@ -664,7 +665,7 @@ dsn::error_code server_state::sync_apps_from_remote_storage()
                         if (app->status == app_status::AS_AVAILABLE) {
                             app->status = app_status::AS_CREATING;
                             _exist_apps.emplace(app->app_name, app);
-                            _table_metric_entities.create_entity(app->app_id);
+                            _table_metric_entities.create_entity(app->app_id, app->partition_count);
                         } else if (app->status == app_status::AS_DROPPED) {
                             app->status = app_status::AS_DROPPING;
                         } else {
@@ -1164,7 +1165,7 @@ void server_state::create_app(dsn::message_ex *msg)
 
             _all_apps.emplace(app->app_id, app);
             _exist_apps.emplace(request.app_name, app);
-            _table_metric_entities.create_entity(app->app_id);
+            _table_metric_entities.create_entity(app->app_id, app->partition_count);
         }
     }
 
@@ -1394,7 +1395,8 @@ void server_state::recall_app(dsn::message_ex *msg)
                     target_app->helpers->pending_response = msg;
 
                     _exist_apps.emplace(target_app->app_name, target_app);
-                    _table_metric_entities.create_entity(target_app->app_id);
+                    _table_metric_entities.create_entity(target_app->app_id,
+                                                         target_app->partition_count);
                 }
             }
         }
@@ -1619,15 +1621,12 @@ void server_state::update_configuration_locally(
         _config_change_subscriber(_all_apps);
     }
 
-    METRIC_CALL_TABLE_INCREMENT_METHOD(
-        _table_metric_entities, partition_configuration_changes, app.app_id);
+    METRIC_INCREMENT(_table_metric_entities, partition_configuration_changes, gpid);
     if (old_health_status >= HS_WRITABLE_ILL && new_health_status < HS_WRITABLE_ILL) {
-        METRIC_CALL_TABLE_INCREMENT_METHOD(
-            _table_metric_entities, unwritable_partition_changes, app.app_id);
+        METRIC_INCREMENT(_table_metric_entities, unwritable_partition_changes, gpid);
     }
     if (old_health_status < HS_WRITABLE_ILL && new_health_status >= HS_WRITABLE_ILL) {
-        METRIC_CALL_TABLE_INCREMENT_METHOD(
-            _table_metric_entities, writable_partition_changes, app.app_id);
+        METRIC_INCREMENT(_table_metric_entities, writable_partition_changes, gpid);
     }
 }
 
@@ -2431,18 +2430,13 @@ void server_state::update_partition_metrics()
             counters[st]++;
         }
 
-        METRIC_CALL_TABLE_SET_METHOD(
-            _table_metric_entities, dead_partitions, app->app_id, counters[HS_DEAD]);
-        METRIC_CALL_TABLE_SET_METHOD(
-            _table_metric_entities, unreadable_partitions, app->app_id, counters[HS_UNREADABLE]);
-        METRIC_CALL_TABLE_SET_METHOD(
-            _table_metric_entities, unwritable_partitions, app->app_id, counters[HS_UNWRITABLE]);
-        METRIC_CALL_TABLE_SET_METHOD(_table_metric_entities,
-                                     writable_ill_partitions,
-                                     app->app_id,
-                                     counters[HS_WRITABLE_ILL]);
-        METRIC_CALL_TABLE_SET_METHOD(
-            _table_metric_entities, healthy_partitions, app->app_id, counters[HS_HEALTHY]);
+        METRIC_SET_TABLE_HEALTH_STATS(_table_metric_entities,
+                                      app->app_id,
+                                      counters[HS_DEAD],
+                                      counters[HS_UNREADABLE],
+                                      counters[HS_UNWRITABLE],
+                                      counters[HS_WRITABLE_ILL],
+                                      counters[HS_HEALTHY]);
 
         return true;
     };
diff --git a/src/meta/server_state.h b/src/meta/server_state.h
index 13a8ef8bd..730ac8a52 100644
--- a/src/meta/server_state.h
+++ b/src/meta/server_state.h
@@ -222,6 +222,8 @@ public:
     task_tracker *tracker() { return &_tracker; }
     void wait_all_task() { _tracker.wait_outstanding_tasks(); }
 
+    table_metric_entities &get_table_metric_entities() { return _table_metric_entities; }
+
 private:
     FRIEND_TEST(backup_service_test, test_invalid_backup_request);
 
diff --git a/src/meta/server_state_restore.cpp b/src/meta/server_state_restore.cpp
index f513b0c4a..b2dada4a2 100644
--- a/src/meta/server_state_restore.cpp
+++ b/src/meta/server_state_restore.cpp
@@ -148,7 +148,7 @@ std::pair<dsn::error_code, std::shared_ptr<app_state>> server_state::restore_app
 
             _all_apps.emplace(app->app_id, app);
             _exist_apps.emplace(info.app_name, app);
-            _table_metric_entities.create_entity(app->app_id);
+            _table_metric_entities.create_entity(app->app_id, app->partition_count);
         }
     }
     // TODO: using one single env to replace
diff --git a/src/meta/table_metrics.cpp b/src/meta/table_metrics.cpp
index 511be89eb..73b992062 100644
--- a/src/meta/table_metrics.cpp
+++ b/src/meta/table_metrics.cpp
@@ -17,12 +17,54 @@
 
 #include "table_metrics.h"
 
+// IWYU pragma: no_include <ext/alloc_traits.h>
 #include <fmt/core.h>
+#include <fmt/ostream.h>
+#include <stddef.h>
+#include <iosfwd>
 #include <string>
 
 #include "utils/fmt_logging.h"
 #include "utils/string_view.h"
 
+METRIC_DEFINE_entity(partition);
+
+METRIC_DEFINE_counter(partition,
+                      partition_configuration_changes,
+                      dsn::metric_unit::kChanges,
+                      "The number of times the configuration has been changed");
+
+METRIC_DEFINE_counter(partition,
+                      unwritable_partition_changes,
+                      dsn::metric_unit::kChanges,
+                      "The number of times the status of partition has been changed to unwritable");
+
+METRIC_DEFINE_counter(partition,
+                      writable_partition_changes,
+                      dsn::metric_unit::kChanges,
+                      "The number of times the status of partition has been changed to writable");
+
+METRIC_DEFINE_gauge_int64(
+    partition,
+    greedy_recent_balance_operations,
+    dsn::metric_unit::kOperations,
+    "The number of balance operations by greedy balancer that are recently needed to be executed");
+
+METRIC_DEFINE_counter(partition,
+                      greedy_move_primary_operations,
+                      dsn::metric_unit::kOperations,
+                      "The number of balance operations by greedy balancer that move primaries");
+
+METRIC_DEFINE_counter(partition,
+                      greedy_copy_primary_operations,
+                      dsn::metric_unit::kOperations,
+                      "The number of balance operations by greedy balancer that copy primaries");
+
+METRIC_DEFINE_counter(partition,
+                      greedy_copy_secondary_operations,
+                      dsn::metric_unit::kOperations,
+                      "The number of balance operations by greedy balancer that copy secondaries");
+
 METRIC_DEFINE_entity(table);
 
 // The number of partitions in each status, see `health_status` and `partition_health_status()`
@@ -59,25 +101,19 @@ METRIC_DEFINE_gauge_int64(table,
                           "The number of healthy partitions, which means primary = 1 && "
                           "primary + secondary >= max_replica_count");
 
-METRIC_DEFINE_counter(table,
-                      partition_configuration_changes,
-                      dsn::metric_unit::kChanges,
-                      "The number of times the configuration has been changed");
-
-METRIC_DEFINE_counter(table,
-                      unwritable_partition_changes,
-                      dsn::metric_unit::kChanges,
-                      "The number of times the status of partition has been changed to unwritable");
-
-METRIC_DEFINE_counter(table,
-                      writable_partition_changes,
-                      dsn::metric_unit::kChanges,
-                      "The number of times the status of partition has been changed to writable");
-
 namespace dsn {
 
 namespace {
 
+metric_entity_ptr instantiate_partition_metric_entity(int32_t table_id, int32_t partition_id)
+{
+    auto entity_id = fmt::format("partition_{}", gpid(table_id, partition_id));
+
+    return METRIC_ENTITY_partition.instantiate(
+        entity_id,
+        {{"table_id", std::to_string(table_id)}, {"partition_id", std::to_string(partition_id)}});
+}
+
 metric_entity_ptr instantiate_table_metric_entity(int32_t table_id)
 {
     auto entity_id = fmt::format("table_{}", table_id);
@@ -87,18 +123,63 @@ metric_entity_ptr instantiate_table_metric_entity(int32_t table_id)
 
 } // anonymous namespace
 
-table_metrics::table_metrics(int32_t table_id)
+partition_metrics::partition_metrics(int32_t table_id, int32_t partition_id)
+    : _table_id(table_id),
+      _partition_id(partition_id),
+      _partition_metric_entity(instantiate_partition_metric_entity(table_id, partition_id)),
+      METRIC_VAR_INIT_partition(partition_configuration_changes),
+      METRIC_VAR_INIT_partition(unwritable_partition_changes),
+      METRIC_VAR_INIT_partition(writable_partition_changes),
+      METRIC_VAR_INIT_partition(greedy_recent_balance_operations),
+      METRIC_VAR_INIT_partition(greedy_move_primary_operations),
+      METRIC_VAR_INIT_partition(greedy_copy_primary_operations),
+      METRIC_VAR_INIT_partition(greedy_copy_secondary_operations)
+{
+}
+
+const metric_entity_ptr &partition_metrics::partition_metric_entity() const
+{
+    CHECK_NOTNULL(_partition_metric_entity,
+                  "partition metric entity should has been instantiated: "
+                  "uninitialized entity cannot be used to instantiate "
+                  "metric");
+    return _partition_metric_entity;
+}
+
+bool operator==(const partition_metrics &lhs, const partition_metrics &rhs)
+{
+    if (&lhs == &rhs) {
+        return true;
+    }
+
+    if (lhs.partition_metric_entity().get() != rhs.partition_metric_entity().get()) {
+        CHECK_TRUE(lhs.table_id() != rhs.table_id() || lhs.partition_id() != rhs.partition_id());
+        return false;
+    }
+
+    CHECK_EQ(lhs.table_id(), rhs.table_id());
+    CHECK_EQ(lhs.partition_id(), rhs.partition_id());
+    return true;
+}
+
+bool operator!=(const partition_metrics &lhs, const partition_metrics &rhs)
+{
+    return !(lhs == rhs);
+}
+
+table_metrics::table_metrics(int32_t table_id, int32_t partition_count)
     : _table_id(table_id),
       _table_metric_entity(instantiate_table_metric_entity(table_id)),
       METRIC_VAR_INIT_table(dead_partitions),
       METRIC_VAR_INIT_table(unreadable_partitions),
       METRIC_VAR_INIT_table(unwritable_partitions),
       METRIC_VAR_INIT_table(writable_ill_partitions),
-      METRIC_VAR_INIT_table(healthy_partitions),
-      METRIC_VAR_INIT_table(partition_configuration_changes),
-      METRIC_VAR_INIT_table(unwritable_partition_changes),
-      METRIC_VAR_INIT_table(writable_partition_changes)
+      METRIC_VAR_INIT_table(healthy_partitions)
 {
+    _partition_metrics.reserve(partition_count);
+    for (int32_t i = 0; i < partition_count; ++i) {
+        _partition_metrics.push_back(std::make_unique<partition_metrics>(table_id, i));
+    }
 }
 
 const metric_entity_ptr &table_metrics::table_metric_entity() const
@@ -110,6 +191,28 @@ const metric_entity_ptr &table_metrics::table_metric_entity() const
     return _table_metric_entity;
 }
 
+void table_metrics::resize_partitions(int32_t partition_count)
+{
+    LOG_INFO("resize partitions for table_metrics(table_id={}): old_partition_count={}, "
+             "new_partition_count={}",
+             _table_id,
+             _partition_metrics.size(),
+             partition_count);
+
+    if (_partition_metrics.size() == partition_count) {
+        return;
+    }
+
+    if (_partition_metrics.size() > partition_count) {
+        _partition_metrics.resize(partition_count);
+        return;
+    }
+
+    for (int32_t i = _partition_metrics.size(); i < partition_count; ++i) {
+        _partition_metrics.push_back(std::make_unique<partition_metrics>(_table_id, i));
+    }
+}
+
 bool operator==(const table_metrics &lhs, const table_metrics &rhs)
 {
     if (&lhs == &rhs) {
@@ -122,33 +225,74 @@ bool operator==(const table_metrics &lhs, const table_metrics &rhs)
     }
 
     CHECK_EQ(lhs.table_id(), rhs.table_id());
+
+    if (lhs._partition_metrics.size() != rhs._partition_metrics.size()) {
+        return false;
+    }
+
+    for (size_t i = 0; i < lhs._partition_metrics.size(); ++i) {
+        if (*(lhs._partition_metrics[i]) != *(rhs._partition_metrics[i])) {
+            return false;
+        }
+    }
+
     return true;
 }
 
 bool operator!=(const table_metrics &lhs, const table_metrics &rhs) { return !(lhs == rhs); }
 
-void table_metric_entities::create_entity(int32_t table_id)
+void table_metric_entities::create_entity(int32_t table_id, int32_t partition_count)
 {
+    LOG_INFO("try to create entity for table_metric_entities(table_id={}): partition_count={}",
+             table_id,
+             partition_count);
+
     utils::auto_write_lock l(_lock);
 
     entity_map::const_iterator iter = _entities.find(table_id);
     if (dsn_unlikely(iter != _entities.end())) {
+        LOG_WARNING("entity has existed for table_metric_entities(table_id={})", table_id);
         return;
     }
 
-    _entities[table_id] = std::make_unique<table_metrics>(table_id);
+    _entities[table_id] = std::make_unique<table_metrics>(table_id, partition_count);
+    LOG_INFO("entity has been created for table_metric_entities(table_id={}): partition_count={}",
+             table_id,
+             partition_count);
+}
+
+void table_metric_entities::resize_partitions(int32_t table_id, int32_t partition_count)
+{
+    LOG_INFO(
+        "try to resize partitions for table_metric_entities(table_id={}): new_partition_count={}",
+        table_id,
+        partition_count);
+
+    utils::auto_write_lock l(_lock);
+
+    auto iter = _entities.find(table_id);
+    if (dsn_unlikely(iter == _entities.end())) {
+        LOG_WARNING("entity does not exist for table_metric_entities(table_id={})", table_id);
+        return;
+    }
+
+    iter->second->resize_partitions(partition_count);
 }
 
 void table_metric_entities::remove_entity(int32_t table_id)
 {
+    LOG_INFO("try to remove entity for table_metric_entities(table_id={})", table_id);
+
     utils::auto_write_lock l(_lock);
 
     entity_map::const_iterator iter = _entities.find(table_id);
     if (dsn_unlikely(iter == _entities.end())) {
+        LOG_WARNING("entity does not exist for table_metric_entities(table_id={})", table_id);
         return;
     }
 
     _entities.erase(iter);
+    LOG_INFO("entity has been removed for table_metric_entities(table_id={})", table_id);
 }
 
 void table_metric_entities::clear_entities()
@@ -157,6 +301,59 @@ void table_metric_entities::clear_entities()
     _entities.clear();
 }
 
+void table_metric_entities::set_health_stats(int32_t table_id,
+                                             int dead_partitions,
+                                             int unreadable_partitions,
+                                             int unwritable_partitions,
+                                             int writable_ill_partitions,
+                                             int healthy_partitions)
+{
+    utils::auto_read_lock l(_lock);
+
+    auto iter = _entities.find(table_id);
+    if (dsn_unlikely(iter == _entities.end())) {
+        return;
+    }
+
+#define __METRIC_SET(name) METRIC_SET(*(iter->second), name, name)
+
+    __METRIC_SET(dead_partitions);
+    __METRIC_SET(unreadable_partitions);
+    __METRIC_SET(unwritable_partitions);
+    __METRIC_SET(writable_ill_partitions);
+    __METRIC_SET(healthy_partitions);
+
+#undef __METRIC_SET
+}
+
+void table_metric_entities::set_greedy_balance_stats(const greedy_balance_stats &balance_stats)
+{
+    utils::auto_read_lock l(_lock);
+
+    const auto &stats = balance_stats.stats();
+    for (const auto &partition : stats) {
+        auto iter = _entities.find(partition.first.get_app_id());
+        if (dsn_unlikely(iter == _entities.end())) {
+            continue;
+        }
+
+        METRIC_SET(*(iter->second),
+                   greedy_recent_balance_operations,
+                   partition.first.get_partition_index(),
+                   partition.second.greedy_recent_balance_operations);
+
+#define __METRIC_INCREMENT_BY(name)                                                                \
+    METRIC_INCREMENT_BY(                                                                           \
+        *(iter->second), name, partition.first.get_partition_index(), partition.second.name)
+
+        __METRIC_INCREMENT_BY(greedy_move_primary_operations);
+        __METRIC_INCREMENT_BY(greedy_copy_primary_operations);
+        __METRIC_INCREMENT_BY(greedy_copy_secondary_operations);
+
+#undef __METRIC_INCREMENT_BY
+    }
+}
+
 bool operator==(const table_metric_entities &lhs, const table_metric_entities &rhs)
 {
     if (&lhs == &rhs) {
diff --git a/src/meta/table_metrics.h b/src/meta/table_metrics.h
index de5db3888..39fec611d 100644
--- a/src/meta/table_metrics.h
+++ b/src/meta/table_metrics.h
@@ -21,35 +21,114 @@
 #include <memory>
 #include <unordered_map>
 #include <utility>
+#include <vector>
 
+#include "common/gpid.h"
 #include "utils/autoref_ptr.h"
+#include "utils/fmt_logging.h"
 #include "utils/metrics.h"
 #include "utils/ports.h"
 #include "utils/synchronize.h"
 
 namespace dsn {
-class table_metric_entities;
+
+// Maintain a partition-level metric entity of meta, and all metrics attached to it.
+class partition_metrics
+{
+public:
+    partition_metrics(int32_t table_id, int32_t partition_id);
+    ~partition_metrics() = default;
+
+    inline int32_t table_id() const { return _table_id; }
+    inline int32_t partition_id() const { return _partition_id; }
+    const metric_entity_ptr &partition_metric_entity() const;
+
+    METRIC_DEFINE_INCREMENT(partition_configuration_changes)
+    METRIC_DEFINE_INCREMENT(unwritable_partition_changes)
+    METRIC_DEFINE_INCREMENT(writable_partition_changes)
+
+    METRIC_DEFINE_SET(greedy_recent_balance_operations, int64_t)
+    METRIC_DEFINE_INCREMENT_BY(greedy_move_primary_operations)
+    METRIC_DEFINE_INCREMENT_BY(greedy_copy_primary_operations)
+    METRIC_DEFINE_INCREMENT_BY(greedy_copy_secondary_operations)
+
+private:
+    const int32_t _table_id;
+    const int32_t _partition_id;
+
+    const metric_entity_ptr _partition_metric_entity;
+    METRIC_VAR_DECLARE_counter(partition_configuration_changes);
+    METRIC_VAR_DECLARE_counter(unwritable_partition_changes);
+    METRIC_VAR_DECLARE_counter(writable_partition_changes);
+    METRIC_VAR_DECLARE_gauge_int64(greedy_recent_balance_operations);
+    METRIC_VAR_DECLARE_counter(greedy_move_primary_operations);
+    METRIC_VAR_DECLARE_counter(greedy_copy_primary_operations);
+    METRIC_VAR_DECLARE_counter(greedy_copy_secondary_operations);
+
+    DISALLOW_COPY_AND_ASSIGN(partition_metrics);
+};
+
+bool operator==(const partition_metrics &lhs, const partition_metrics &rhs);
+bool operator!=(const partition_metrics &lhs, const partition_metrics &rhs);
 
 // Maintain a table-level metric entity of meta, and all metrics attached to it.
 class table_metrics
 {
 public:
-    table_metrics(int32_t table_id);
+    table_metrics(int32_t table_id, int32_t partition_count);
     ~table_metrics() = default;
 
     inline int32_t table_id() const { return _table_id; }
     const metric_entity_ptr &table_metric_entity() const;
 
-    METRIC_DEFINE_SET_METHOD(dead_partitions, int64_t)
-    METRIC_DEFINE_SET_METHOD(unreadable_partitions, int64_t)
-    METRIC_DEFINE_SET_METHOD(unwritable_partitions, int64_t)
-    METRIC_DEFINE_SET_METHOD(writable_ill_partitions, int64_t)
-    METRIC_DEFINE_SET_METHOD(healthy_partitions, int64_t)
-    METRIC_DEFINE_INCREMENT_METHOD(partition_configuration_changes)
-    METRIC_DEFINE_INCREMENT_METHOD(unwritable_partition_changes)
-    METRIC_DEFINE_INCREMENT_METHOD(writable_partition_changes)
+    void resize_partitions(int32_t partition_count);
+
+    METRIC_DEFINE_SET(dead_partitions, int64_t)
+    METRIC_DEFINE_SET(unreadable_partitions, int64_t)
+    METRIC_DEFINE_SET(unwritable_partitions, int64_t)
+    METRIC_DEFINE_SET(writable_ill_partitions, int64_t)
+    METRIC_DEFINE_SET(healthy_partitions, int64_t)
+
+#define __METRIC_DEFINE_INCREMENT_BY(name)                                                         \
+    void increment_##name##_by(int32_t partition_id, int64_t x)                                    \
+    {                                                                                              \
+        CHECK_LT(partition_id, _partition_metrics.size());                                         \
+        METRIC_INCREMENT_BY(*(_partition_metrics[partition_id]), name, x);                         \
+    }
+
+    __METRIC_DEFINE_INCREMENT_BY(greedy_move_primary_operations)
+    __METRIC_DEFINE_INCREMENT_BY(greedy_copy_primary_operations)
+    __METRIC_DEFINE_INCREMENT_BY(greedy_copy_secondary_operations)
+
+#undef __METRIC_DEFINE_INCREMENT_BY
+
+#define __METRIC_DEFINE_INCREMENT(name)                                                            \
+    void increment_##name(int32_t partition_id)                                                    \
+    {                                                                                              \
+        CHECK_LT(partition_id, _partition_metrics.size());                                         \
+        METRIC_INCREMENT(*(_partition_metrics[partition_id]), name);                               \
+    }
+
+    __METRIC_DEFINE_INCREMENT(partition_configuration_changes)
+    __METRIC_DEFINE_INCREMENT(unwritable_partition_changes)
+    __METRIC_DEFINE_INCREMENT(writable_partition_changes)
+
+#undef __METRIC_DEFINE_INCREMENT
+
+#define __METRIC_DEFINE_SET(name, value_type)                                                      \
+    void set_##name(int32_t partition_id, value_type value)                                        \
+    {                                                                                              \
+        CHECK_LT(partition_id, _partition_metrics.size());                                         \
+        METRIC_SET(*(_partition_metrics[partition_id]), name, value);                              \
+    }
+
+    __METRIC_DEFINE_SET(greedy_recent_balance_operations, int64_t)
+
+#undef __METRIC_DEFINE_SET
 
 private:
+    friend bool operator==(const table_metrics &, const table_metrics &);
+
     const int32_t _table_id;
 
     const metric_entity_ptr _table_metric_entity;
@@ -58,9 +137,8 @@ private:
     METRIC_VAR_DECLARE_gauge_int64(unwritable_partitions);
     METRIC_VAR_DECLARE_gauge_int64(writable_ill_partitions);
     METRIC_VAR_DECLARE_gauge_int64(healthy_partitions);
-    METRIC_VAR_DECLARE_counter(partition_configuration_changes);
-    METRIC_VAR_DECLARE_counter(unwritable_partition_changes);
-    METRIC_VAR_DECLARE_counter(writable_partition_changes);
+
+    std::vector<std::unique_ptr<partition_metrics>> _partition_metrics;
 
     DISALLOW_COPY_AND_ASSIGN(table_metrics);
 };
@@ -68,33 +146,46 @@ private:
 bool operator==(const table_metrics &lhs, const table_metrics &rhs);
 bool operator!=(const table_metrics &lhs, const table_metrics &rhs);
 
-#define METRIC_DEFINE_TABLE_SET_METHOD(name, value_type)                                           \
-    void set_##name(int32_t table_id, value_type value)                                            \
-    {                                                                                              \
-        utils::auto_read_lock l(_lock);                                                            \
-                                                                                                   \
-        entity_map::const_iterator iter = _entities.find(table_id);                                \
-        if (dsn_unlikely(iter == _entities.end())) {                                               \
-            return;                                                                                \
-        }                                                                                          \
-        METRIC_CALL_SET_METHOD(*(iter->second), name, value);                                      \
-    }
+class greedy_balance_stats
+{
+public:
+    greedy_balance_stats() = default;
+    ~greedy_balance_stats() = default;
+
+    struct partition_stats
+    {
+        int greedy_recent_balance_operations = 0;
+        int greedy_move_primary_operations = 0;
+        int greedy_copy_primary_operations = 0;
+        int greedy_copy_secondary_operations = 0;
+    };
 
-#define METRIC_CALL_TABLE_SET_METHOD(obj, name, table_id, value) (obj).set_##name(table_id, value)
+    using partition_map = std::unordered_map<gpid, partition_stats>;
 
-#define METRIC_DEFINE_TABLE_INCREMENT_METHOD(name)                                                 \
-    void increment_##name(int32_t table_id)                                                        \
+#define __METRIC_DEFINE_INCREMENT(name)                                                            \
+    void increment_##name(const gpid &id, bool balance_checker)                                    \
     {                                                                                              \
-        utils::auto_read_lock l(_lock);                                                            \
-                                                                                                   \
-        entity_map::const_iterator iter = _entities.find(table_id);                                \
-        if (dsn_unlikely(iter == _entities.end())) {                                               \
+        auto &partition = _partition_map[id];                                                      \
+        ++(partition.greedy_recent_balance_operations);                                            \
+        if (balance_checker) {                                                                     \
             return;                                                                                \
         }                                                                                          \
-        METRIC_CALL_INCREMENT_METHOD(*(iter->second), name);                                       \
+        ++(partition.name);                                                                        \
     }
 
-#define METRIC_CALL_TABLE_INCREMENT_METHOD(obj, name, table_id) (obj).increment_##name(table_id)
+    __METRIC_DEFINE_INCREMENT(greedy_move_primary_operations)
+    __METRIC_DEFINE_INCREMENT(greedy_copy_primary_operations)
+    __METRIC_DEFINE_INCREMENT(greedy_copy_secondary_operations)
+
+#undef __METRIC_DEFINE_INCREMENT
+
+    const partition_map &stats() const { return _partition_map; }
+
+private:
+    partition_map _partition_map;
+
+    DISALLOW_COPY_AND_ASSIGN(greedy_balance_stats);
+};
 
 // Manage the lifetime of all table-level metric entities of meta.
 //
@@ -109,18 +200,38 @@ public:
     table_metric_entities() = default;
     ~table_metric_entities() = default;
 
-    void create_entity(int32_t table_id);
+    void create_entity(int32_t table_id, int32_t partition_count);
+    void resize_partitions(int32_t table_id, int32_t partition_count);
     void remove_entity(int32_t table_id);
     void clear_entities();
 
-    METRIC_DEFINE_TABLE_SET_METHOD(dead_partitions, int64_t)
-    METRIC_DEFINE_TABLE_SET_METHOD(unreadable_partitions, int64_t)
-    METRIC_DEFINE_TABLE_SET_METHOD(unwritable_partitions, int64_t)
-    METRIC_DEFINE_TABLE_SET_METHOD(writable_ill_partitions, int64_t)
-    METRIC_DEFINE_TABLE_SET_METHOD(healthy_partitions, int64_t)
-    METRIC_DEFINE_TABLE_INCREMENT_METHOD(partition_configuration_changes)
-    METRIC_DEFINE_TABLE_INCREMENT_METHOD(unwritable_partition_changes)
-    METRIC_DEFINE_TABLE_INCREMENT_METHOD(writable_partition_changes)
+#define __METRIC_DEFINE_INCREMENT(name)                                                            \
+    void increment_##name(const gpid &id)                                                          \
+    {                                                                                              \
+        utils::auto_read_lock l(_lock);                                                            \
+                                                                                                   \
+        auto iter = _entities.find(id.get_app_id());                                               \
+        if (dsn_unlikely(iter == _entities.end())) {                                               \
+            return;                                                                                \
+        }                                                                                          \
+                                                                                                   \
+        METRIC_INCREMENT(*(iter->second), name, id.get_partition_index());                         \
+    }
+
+    __METRIC_DEFINE_INCREMENT(partition_configuration_changes)
+    __METRIC_DEFINE_INCREMENT(unwritable_partition_changes)
+    __METRIC_DEFINE_INCREMENT(writable_partition_changes)
+
+#undef __METRIC_DEFINE_INCREMENT
+
+    void set_greedy_balance_stats(const greedy_balance_stats &balance_stats);
+
+    void set_health_stats(int32_t table_id,
+                          int dead_partitions,
+                          int unreadable_partitions,
+                          int unwritable_partitions,
+                          int writable_ill_partitions,
+                          int healthy_partitions);
 
 private:
     friend bool operator==(const table_metric_entities &, const table_metric_entities &);
@@ -133,4 +244,9 @@ private:
 
 bool operator==(const table_metric_entities &lhs, const table_metric_entities &rhs);
 
+#define METRIC_SET_GREEDY_BALANCE_STATS(obj, ...) (obj).set_greedy_balance_stats(__VA_ARGS__)
+
+#define METRIC_SET_TABLE_HEALTH_STATS(obj, table_id, ...)                                          \
+    (obj).set_health_stats(table_id, ##__VA_ARGS__)
+
 } // namespace dsn
diff --git a/src/meta/test/backup_test.cpp b/src/meta/test/backup_test.cpp
index 70062ba69..35b88bb97 100644
--- a/src/meta/test/backup_test.cpp
+++ b/src/meta/test/backup_test.cpp
@@ -29,6 +29,7 @@
 #include <set>
 #include <string>
 #include <thread>
+#include <type_traits>
 #include <utility>
 #include <vector>
 
diff --git a/src/meta/test/meta_split_service_test.cpp b/src/meta/test/meta_split_service_test.cpp
index ea611c633..762987d78 100644
--- a/src/meta/test/meta_split_service_test.cpp
+++ b/src/meta/test/meta_split_service_test.cpp
@@ -55,6 +55,7 @@
 #include "meta/meta_split_service.h"
 #include "meta/meta_state_service_utils.h"
 #include "meta/server_state.h"
+#include "meta/table_metrics.h"
 #include "meta_admin_types.h"
 #include "meta_service_test_app.h"
 #include "meta_test_base.h"
@@ -208,6 +209,7 @@ public:
     {
         app->partition_count = NEW_PARTITION_COUNT;
         app->partitions.resize(app->partition_count);
+        _ss->get_table_metric_entities().resize_partitions(app->app_id, app->partition_count);
         app->helpers->contexts.resize(app->partition_count);
         app->helpers->split_states.splitting_count = app->partition_count / 2;
         for (int i = 0; i < app->partition_count; ++i) {
@@ -227,6 +229,7 @@ public:
     {
         app->partition_count = PARTITION_COUNT;
         app->partitions.resize(app->partition_count);
+        _ss->get_table_metric_entities().resize_partitions(app->app_id, app->partition_count);
         app->helpers->contexts.resize(app->partition_count);
         app->helpers->split_states.splitting_count = 0;
         app->helpers->split_states.status.clear();
@@ -236,6 +239,7 @@ public:
     {
         app->partition_count = NEW_PARTITION_COUNT;
         app->partitions.resize(app->partition_count);
+        _ss->get_table_metric_entities().resize_partitions(app->app_id, app->partition_count);
         app->helpers->contexts.resize(app->partition_count);
         for (int i = 0; i < app->partition_count; ++i) {
             app->helpers->contexts[i].config_owner = &app->partitions[i];
diff --git a/src/meta/test/update_configuration_test.cpp b/src/meta/test/update_configuration_test.cpp
index cb30bff2b..4e8f34f64 100644
--- a/src/meta/test/update_configuration_test.cpp
+++ b/src/meta/test/update_configuration_test.cpp
@@ -496,7 +496,7 @@ void meta_service_test_app::cannot_run_balancer_test()
     std::shared_ptr<app_state> the_app = app_state::create(info);
     svc->_state->_all_apps.emplace(info.app_id, the_app);
     svc->_state->_exist_apps.emplace(info.app_name, the_app);
-    svc->_state->_table_metric_entities.create_entity(info.app_id);
+    svc->_state->_table_metric_entities.create_entity(info.app_id, info.partition_count);
 
     dsn::partition_configuration &pc = the_app->partitions[0];
     pc.primary = nodes[0];
diff --git a/src/replica/test/replica_test.cpp b/src/replica/test/replica_test.cpp
index fbb872715..695417c86 100644
--- a/src/replica/test/replica_test.cpp
+++ b/src/replica/test/replica_test.cpp
@@ -18,6 +18,7 @@
 // IWYU pragma: no_include <gtest/gtest-message.h>
 // IWYU pragma: no_include <gtest/gtest-test-part.h>
 #include <gtest/gtest.h>
+#include <stddef.h>
 #include <stdint.h>
 #include <iostream>
 #include <map>
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index ba85c3fd4..cb6b7cb48 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -167,6 +167,7 @@ class error_code;
 #define METRIC_VAR_INIT_server(name, ...) METRIC_VAR_INIT(name, server, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_disk(name, ...) METRIC_VAR_INIT(name, disk, ##__VA_ARGS__)
 #define METRIC_VAR_INIT_table(name, ...) METRIC_VAR_INIT(name, table, ##__VA_ARGS__)
+#define METRIC_VAR_INIT_partition(name, ...) METRIC_VAR_INIT(name, partition, ##__VA_ARGS__)
 
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
@@ -196,15 +197,23 @@ class error_code;
 
 #define METRIC_VAR_AUTO_LATENCY_DURATION_NS(name) __##name##_auto_latency.duration_ns()
 
-#define METRIC_DEFINE_SET_METHOD(name, value_type)                                                 \
-    void set_##name(value_type value) { METRIC_VAR_SET(name, value); }
+#define METRIC_DEFINE_INCREMENT_BY(name)                                                           \
+    void increment_##name##_by(int64_t x) { METRIC_VAR_INCREMENT_BY(name, x); }
 
-#define METRIC_CALL_SET_METHOD(obj, name, value) (obj).set_##name(value)
+// To be adaptive to self-defined `increment_by` methods, arguments are declared as variadic.
+#define METRIC_INCREMENT_BY(obj, name, ...) (obj).increment_##name##_by(__VA_ARGS__)
 
-#define METRIC_DEFINE_INCREMENT_METHOD(name)                                                       \
+#define METRIC_DEFINE_INCREMENT(name)                                                              \
     void increment_##name() { METRIC_VAR_INCREMENT(name); }
 
-#define METRIC_CALL_INCREMENT_METHOD(obj, name) (obj).increment_##name()
+// To be adaptive to self-defined `increment` methods, arguments are declared as variadic.
+#define METRIC_INCREMENT(obj, name, ...) (obj).increment_##name(__VA_ARGS__)
+
+#define METRIC_DEFINE_SET(name, value_type)                                                        \
+    void set_##name(value_type value) { METRIC_VAR_SET(name, value); }
+
+// To be adaptive to self-defined `set` methods, arguments are declared as variadic.
+#define METRIC_SET(obj, name, ...) (obj).set_##name(__VA_ARGS__)
 
 namespace dsn {
 class metric;                  // IWYU pragma: keep
@@ -639,6 +648,7 @@ enum class metric_unit : size_t
     kCompactions,
     kWrites,
     kChanges,
+    kOperations,
     kInvalidUnit,
 };
 


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


[incubator-pegasus] 09/23: feat(new_metrics): add server-level metric entity (#1415)

Posted by wa...@apache.org.
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 20b88806ceb56566f05e86405ba10e50bcf226fb
Author: Dan Wang <wa...@apache.org>
AuthorDate: Sat Mar 25 11:24:57 2023 +0800

    feat(new_metrics): add server-level metric entity (#1415)
    
    https://github.com/apache/incubator-pegasus/issues/1414
    
    Add server-level metric entity, including:
    - Define server-level entity prototype in metrics.cpp;
    - Implement getter to acquire the (only) instance of server entity;
    - Provide convenient macros to simplify operations for server entity.
---
 src/utils/metrics.cpp | 8 ++++++++
 src/utils/metrics.h   | 7 +++++++
 2 files changed, 15 insertions(+)

diff --git a/src/utils/metrics.cpp b/src/utils/metrics.cpp
index 2a15fb5aa..b32887c55 100644
--- a/src/utils/metrics.cpp
+++ b/src/utils/metrics.cpp
@@ -30,6 +30,14 @@
 #include "utils/string_conv.h"
 #include "utils/strings.h"
 
+METRIC_DEFINE_entity(server);
+
+dsn::metric_entity_ptr server_metric_entity()
+{
+    static auto entity = METRIC_ENTITY_server.instantiate("server");
+    return entity;
+}
+
 namespace dsn {
 
 DSN_DEFINE_uint64(metrics,
diff --git a/src/utils/metrics.h b/src/utils/metrics.h
index 9087b78e5..e69268006 100644
--- a/src/utils/metrics.h
+++ b/src/utils/metrics.h
@@ -164,6 +164,7 @@ class error_code;
 #define METRIC_VAR_INIT(name, entity, ...)                                                         \
     _##name(METRIC_##name.instantiate(entity##_metric_entity(), ##__VA_ARGS__))
 #define METRIC_VAR_INIT_replica(name, ...) METRIC_VAR_INIT(name, replica, ##__VA_ARGS__)
+#define METRIC_VAR_INIT_server(name, ...) METRIC_VAR_INIT(name, server, ##__VA_ARGS__)
 
 // Perform increment-related operations on metrics including gauge and counter.
 #define METRIC_VAR_INCREMENT_BY(name, x)                                                           \
@@ -1445,3 +1446,9 @@ private:
 };
 
 } // namespace dsn
+
+// Since server_metric_entity() will be called in macros such as METRIC_VAR_INIT_server(), its
+// declaration should be put outside any namespace (for example dsn). server_metric_entity()
+// will not be qualified with any namespace. Once it was qualified with some namespace, its name
+// would not be resolved in any other namespace.
+dsn::metric_entity_ptr server_metric_entity();


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