You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2020/07/18 15:04:12 UTC

[GitHub] [incubator-doris] acelyc111 opened a new pull request #4115: [metrics] Redesign metrics to 3 layers

acelyc111 opened a new pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115


   Redesign metrics to 3 layers:
   MetricRegistry - MetricEntity - Metrics
   MetricRegistry: the register center
   MetricEntity: the enity registered on MetricRegistry. Generally a MetricRegistry
   can be registered on serveral MetricEntities, each of MetricEntity is an
   independent entity, such as server, disk_devices, data_directories,
   thrift clients and servers, and so on.
   Metric: metrics of an entity. Such as fragment_requests_total on server
   entity, disk_bytes_read on a disk_device entity, thrift_opened_clients
   on a thrift_client entity.
   MetricPrototype: the type of a metric. MetricPrototype is a global
   variable, can be shared by the same metrics across different
   MetricEntities.
   
   The output of /metrics interface also keep compatiable with old versions.
   
   ## Proposed changes
   
   Describe the big picture of your changes here to communicate to the maintainers why we should accept this pull request. If it fixes a bug or resolves a feature request, be sure to link to that issue.
   
   ## Types of changes
   
   What types of changes does your code introduce to Doris?
   _Put an `x` in the boxes that apply_
   
   - [ ] Bugfix (non-breaking change which fixes an issue)
   - [ ] New feature (non-breaking change which adds functionality)
   - [ ] Breaking change (fix or feature that would cause existing functionality to not work as expected)
   - [ ] Documentation Update (if none of the other choices apply)
   
   ## Checklist
   
   _Put an `x` in the boxes that apply. You can also fill these out after creating the PR. If you're unsure about any of them, don't hesitate to ask. We're here to help! This is simply a reminder of what we are going to look for before merging your code._
   
   - [ ] I have create an issue on [Doris's issues](https://github.com/apache/incubator-doris/issues), and have described the bug/feature there in detail
   - [ ] Commit messages in my PR start with the related issues ID, like "#4071 Add pull request template to doris project"
   - [ ] Compiling and unit tests pass locally with my changes
   - [ ] I have added tests that prove my fix is effective or that my feature works
   - [ ] If this change need a document change, I have updated the document
   - [ ] Any dependent changes have been merged
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at dev@doris.apache.org by explaining why you chose the solution you did and what alternatives you considered, etc...


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] morningman commented on pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
morningman commented on pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115#issuecomment-663055507


   I took a cursory look at the code and found no problems.
   This CL is too big and involves too many modules...
   Have you run it in your product env yet?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] chaoyli merged pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
chaoyli merged pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115#discussion_r466937847



##########
File path: be/src/util/metrics.h
##########
@@ -193,230 +174,163 @@ class CoreLocalCounter : public Metric {
     void increment(const T& delta) {
         __sync_fetch_and_add(_value.access(), delta);
     }
+
+    rj::Value to_json_value() const override {
+        return rj::Value(value());
+    }
+
 protected:
     CoreLocalValue<T> _value;
 };
 
 template<typename T>
 class AtomicCounter : public AtomicMetric<T> {
 public:
-    AtomicCounter(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~AtomicCounter() { }
+    AtomicCounter() {}
+    virtual ~AtomicCounter() {}
 };
 
 template<typename T>
 class AtomicGauge : public AtomicMetric<T> {
 public:
-    AtomicGauge(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~AtomicGauge() { }
+    AtomicGauge() : AtomicMetric<T>() {}
+    virtual ~AtomicGauge() {}
 };
 
 template<typename T>
 class LockCounter : public LockSimpleMetric<T> {
 public:
-    LockCounter(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~LockCounter() { }
+    LockCounter() : LockSimpleMetric<T>() {}
+    virtual ~LockCounter() {}
 };
 
 // This can only used for trival type
 template<typename T>
 class LockGauge : public LockSimpleMetric<T> {
 public:
-    LockGauge(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~LockGauge() { }
+    LockGauge() : LockSimpleMetric<T>() {}
+    virtual ~LockGauge() {}
 };
 
-// one key-value pair used to
-struct MetricLabel {
+using Labels = std::unordered_map<std::string, std::string>;
+struct MetricPrototype {
+public:
+    MetricPrototype(MetricType type_,
+                    MetricUnit unit_,
+                    std::string name_,
+                    std::string description_ = "",
+                    std::string group_name_ = "",
+                    Labels labels_ = Labels(),
+                    bool is_core_metric_ = false)
+        : is_core_metric(is_core_metric_),
+          type(type_),
+          unit(unit_),
+          name(std::move(name_)),
+          description(std::move(description_)),
+          group_name(std::move(group_name_)),
+          labels(std::move(labels_)) {}
+
+    std::string simple_name() const;
+    std::string combine_name(const std::string& registry_name) const;
+
+    bool is_core_metric;
+    MetricType type;
+    MetricUnit unit;
     std::string name;
-    std::string value;
+    std::string description;
+    std::string group_name;
+    Labels labels;
+};
 
-    MetricLabel() { }
-    MetricLabel(const std::string& name_, const std::string& value_) :name(name_), value(value_) {
-    }
+#define DEFINE_METRIC(name, type, unit, desc, group, labels, core)      \
+    ::doris::MetricPrototype METRIC_##name(type, unit, #name, desc, group, labels, core)
 
-    bool operator==(const MetricLabel& other) const {
-        return name == other.name && value == other.value;
-    }
-    bool operator!=(const MetricLabel& other) const {
-        return !(*this == other);
-    }
-    bool operator<(const MetricLabel& other) const {
-        auto res = name.compare(other.name);
-        if (res == 0) {
-            return value < other.value;
-        }
-        return res < 0;
-    }
-    int compare(const MetricLabel& other) const {
-        auto res = name.compare(other.name);
-        if (res == 0) {
-            return value.compare(other.value);
-        }
-        return res;
-    }
-    std::string to_string() const {
-        return name + "=" + value;
-    }
-};
+#define DEFINE_COUNTER_METRIC_2ARG(name, unit)                          \
+    DEFINE_METRIC(name, MetricType::COUNTER, unit, "", "", Labels(), false)
 
-struct MetricLabels {
-    static MetricLabels EmptyLabels;
-    // used std::set to sort MetricLabel so that we can get compare two MetricLabels
-    std::set<MetricLabel> labels;
+#define DEFINE_COUNTER_METRIC_3ARG(name, unit, desc)                    \
+    DEFINE_METRIC(name, MetricType::COUNTER, unit, desc, "", Labels(), false)
 
-    MetricLabels& add(const std::string& name, const std::string& value) {
-        labels.emplace(name, value);
-        return *this;
-    }
+#define DEFINE_COUNTER_METRIC_5ARG(name, unit, desc, group, labels)     \
+    DEFINE_METRIC(name, MetricType::COUNTER, unit, desc, #group, labels, false)
 
-    bool operator==(const MetricLabels& other) const {
-        if (labels.size() != other.labels.size()) {
-            return false;
-        }
-        auto it = labels.begin();
-        auto other_it = other.labels.begin();
-        while (it != labels.end()) {
-            if (*it != *other_it) {
-                return false;
-            }
-            ++it;
-            ++other_it;
-        }
-        return true;
-    }
-    bool operator<(const MetricLabels& other) const {
-        auto it = labels.begin();
-        auto other_it = other.labels.begin();
-        while (it != labels.end() && other_it != other.labels.end()) {
-            auto res = it->compare(*other_it);
-            if (res < 0) {
-                return true;
-            } else if (res > 0) {
-                return false;
-            }
-            ++it;
-            ++other_it;
-        }
-        if (it == labels.end()) {
-            if (other_it == other.labels.end()) {
-                return false;
-            }
-            return true;
-        } else {
-            return false;
-        }
-    }
-    bool empty() const {
-        return labels.empty();
+#define DEFINE_GAUGE_METRIC_2ARG(name, unit)                            \
+    DEFINE_METRIC(name, MetricType::GAUGE, unit, "", "", Labels(), false)
+
+#define DEFINE_CORE_GAUGE_METRIC_2ARG(name, unit)                       \
+    DEFINE_METRIC(name, MetricType::GAUGE, unit, "", "", Labels(), true)
+
+#define DEFINE_GAUGE_METRIC_3ARG(name, unit, desc)                      \
+    DEFINE_METRIC(name, MetricType::GAUGE, unit, desc, "", Labels(), false)
+
+#define METRIC_REGISTER(entity, metric)                                 \
+    entity->register_metric(&METRIC_##metric, &metric)
+
+#define METRIC_DEREGISTER(entity, metric)                               \
+    entity->deregister_metric(&METRIC_##metric)
+
+// For 'metrics' in MetricEntity.
+struct MetricPrototypeHash {
+    size_t operator()(const MetricPrototype* metric_prototype) const {
+        return std::hash<std::string>()(metric_prototype->group_name.empty() ? metric_prototype->name : metric_prototype->group_name);
     }
+};
 
-    std::string to_string() const {
-        std::stringstream ss;
-        int i = 0;
-        for (auto& label : labels) {
-            if (i++ > 0) {
-                ss << ",";
-            }
-            ss << label.to_string();
-        }
-        return ss.str();
+struct MetricPrototypeEqualTo {
+    bool operator()(const MetricPrototype* first, const MetricPrototype* second) const {
+        return first->group_name == second->group_name && first->name == second->name;
     }
 };
 
-class MetricCollector;
+using MetricByType = std::unordered_map<const MetricPrototype*, Metric*, MetricPrototypeHash, MetricPrototypeEqualTo>;

Review comment:
       MetricMap would be better




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] acelyc111 commented on pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115#issuecomment-665470640


   > I took a cursory look at the code and found no problems.
   > This CL is too big and involves too many modules...
   > Have you run it in your product env yet?
   
   Yes, I've run it in our product env, it works well.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] morningman commented on pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
morningman commented on pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115#issuecomment-670286448


   I've test it in our env and it looks good to me.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] morningman commented on a change in pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115#discussion_r459497028



##########
File path: be/src/util/doris_metrics.h
##########
@@ -28,163 +28,152 @@
 
 namespace doris {
 
-class IntGaugeMetricsMap {
-public:
-    void set_metric(const std::string& key, int64_t val) {
-        auto metric = metrics.find(key);
-        if (metric != metrics.end()) {
-            metric->second->set_value(val);
-        }
-    }
-
-    IntGauge* add_metric(const std::string& key, const MetricUnit unit) {
-        metrics.emplace(key, new IntGauge(unit));
-        return metrics.find(key)->second.get();
-    }
-
-private:
-    std::unordered_map<std::string, std::unique_ptr<IntGauge>> metrics;
-};
-
-#define REGISTER_GAUGE_DORIS_METRIC(name, func) \
-  DorisMetrics::instance()->metrics()->register_metric(#name, &DorisMetrics::instance()->name); \
-  DorisMetrics::instance()->metrics()->register_hook(#name, [&]() { \
+#define REGISTER_HOOK_METRIC(name, func) \
+  DorisMetrics::instance()->server_entity()->register_metric(&METRIC_##name, &DorisMetrics::instance()->name); \
+  DorisMetrics::instance()->server_entity()->register_hook(#name, [&]() { \
       DorisMetrics::instance()->name.set_value(func());  \
 });
 
+#define DEREGISTER_HOOK_METRIC(name) \
+  DorisMetrics::instance()->server_entity()->deregister_metric(&METRIC_##name); \
+  DorisMetrics::instance()->server_entity()->deregister_hook(#name);
+
 class DorisMetrics {
 public:
-    // counters
-    METRIC_DEFINE_INT_COUNTER(fragment_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(fragment_request_duration_us, MetricUnit::MICROSECONDS);
-    METRIC_DEFINE_INT_COUNTER(http_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(http_request_send_bytes, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(query_scan_bytes, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(query_scan_rows, MetricUnit::ROWS);
-    METRIC_DEFINE_INT_COUNTER(push_requests_success_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(push_requests_fail_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(push_request_duration_us, MetricUnit::MICROSECONDS);
-    METRIC_DEFINE_INT_COUNTER(push_request_write_bytes, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(push_request_write_rows, MetricUnit::ROWS);
-    METRIC_DEFINE_INT_COUNTER(create_tablet_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(create_tablet_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(drop_tablet_requests_total, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(report_all_tablets_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_all_tablets_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_tablet_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_tablet_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_disk_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_disk_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_task_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_task_requests_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(schema_change_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(schema_change_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(create_rollup_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(create_rollup_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(storage_migrate_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(delete_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(delete_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(clone_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(clone_requests_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(finish_task_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(finish_task_requests_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(base_compaction_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(base_compaction_request_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_request_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(base_compaction_deltas_total, MetricUnit::ROWSETS);
-    METRIC_DEFINE_INT_COUNTER(base_compaction_bytes_total, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_deltas_total, MetricUnit::ROWSETS);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_bytes_total, MetricUnit::BYTES);
-
-    METRIC_DEFINE_INT_COUNTER(publish_task_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(publish_task_failed_total, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(meta_write_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(meta_write_request_duration_us, MetricUnit::MICROSECONDS);
-    METRIC_DEFINE_INT_COUNTER(meta_read_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(meta_read_request_duration_us, MetricUnit::MICROSECONDS);
+    //METRIC_DEFINE_INT_COUNTER(load_rows_total, MetricUnit::ROWS);
+    //METRIC_DEFINE_INT_COUNTER(load_bytes_total, MetricUnit::BYTES);
+
+    //IntGaugeMetricsMap disks_total_capacity;

Review comment:
       Remove the unused code

##########
File path: be/src/util/doris_metrics.h
##########
@@ -28,163 +28,152 @@
 
 namespace doris {
 
-class IntGaugeMetricsMap {
-public:
-    void set_metric(const std::string& key, int64_t val) {
-        auto metric = metrics.find(key);
-        if (metric != metrics.end()) {
-            metric->second->set_value(val);
-        }
-    }
-
-    IntGauge* add_metric(const std::string& key, const MetricUnit unit) {
-        metrics.emplace(key, new IntGauge(unit));
-        return metrics.find(key)->second.get();
-    }
-
-private:
-    std::unordered_map<std::string, std::unique_ptr<IntGauge>> metrics;
-};
-
-#define REGISTER_GAUGE_DORIS_METRIC(name, func) \
-  DorisMetrics::instance()->metrics()->register_metric(#name, &DorisMetrics::instance()->name); \
-  DorisMetrics::instance()->metrics()->register_hook(#name, [&]() { \
+#define REGISTER_HOOK_METRIC(name, func) \
+  DorisMetrics::instance()->server_entity()->register_metric(&METRIC_##name, &DorisMetrics::instance()->name); \
+  DorisMetrics::instance()->server_entity()->register_hook(#name, [&]() { \
       DorisMetrics::instance()->name.set_value(func());  \
 });
 
+#define DEREGISTER_HOOK_METRIC(name) \
+  DorisMetrics::instance()->server_entity()->deregister_metric(&METRIC_##name); \
+  DorisMetrics::instance()->server_entity()->deregister_hook(#name);
+
 class DorisMetrics {
 public:
-    // counters
-    METRIC_DEFINE_INT_COUNTER(fragment_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(fragment_request_duration_us, MetricUnit::MICROSECONDS);
-    METRIC_DEFINE_INT_COUNTER(http_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(http_request_send_bytes, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(query_scan_bytes, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(query_scan_rows, MetricUnit::ROWS);
-    METRIC_DEFINE_INT_COUNTER(push_requests_success_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(push_requests_fail_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(push_request_duration_us, MetricUnit::MICROSECONDS);
-    METRIC_DEFINE_INT_COUNTER(push_request_write_bytes, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(push_request_write_rows, MetricUnit::ROWS);
-    METRIC_DEFINE_INT_COUNTER(create_tablet_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(create_tablet_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(drop_tablet_requests_total, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(report_all_tablets_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_all_tablets_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_tablet_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_tablet_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_disk_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_disk_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_task_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(report_task_requests_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(schema_change_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(schema_change_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(create_rollup_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(create_rollup_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(storage_migrate_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(delete_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(delete_requests_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(clone_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(clone_requests_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(finish_task_requests_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(finish_task_requests_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(base_compaction_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(base_compaction_request_failed, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_request_failed, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(base_compaction_deltas_total, MetricUnit::ROWSETS);
-    METRIC_DEFINE_INT_COUNTER(base_compaction_bytes_total, MetricUnit::BYTES);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_deltas_total, MetricUnit::ROWSETS);
-    METRIC_DEFINE_INT_COUNTER(cumulative_compaction_bytes_total, MetricUnit::BYTES);
-
-    METRIC_DEFINE_INT_COUNTER(publish_task_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(publish_task_failed_total, MetricUnit::REQUESTS);
-
-    METRIC_DEFINE_INT_COUNTER(meta_write_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(meta_write_request_duration_us, MetricUnit::MICROSECONDS);
-    METRIC_DEFINE_INT_COUNTER(meta_read_request_total, MetricUnit::REQUESTS);
-    METRIC_DEFINE_INT_COUNTER(meta_read_request_duration_us, MetricUnit::MICROSECONDS);
+    //METRIC_DEFINE_INT_COUNTER(load_rows_total, MetricUnit::ROWS);
+    //METRIC_DEFINE_INT_COUNTER(load_bytes_total, MetricUnit::BYTES);
+
+    //IntGaugeMetricsMap disks_total_capacity;
+    //IntGaugeMetricsMap disks_avail_capacity;
+    //IntGaugeMetricsMap disks_data_used_capacity;
+    //IntGaugeMetricsMap disks_state;
+
+    IntCounter fragment_requests_total;
+	IntCounter fragment_request_duration_us;

Review comment:
       Use 4 space instead of tab




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-doris] chaoyli commented on a change in pull request #4115: [metrics] Redesign metrics to 3 layers

Posted by GitBox <gi...@apache.org>.
chaoyli commented on a change in pull request #4115:
URL: https://github.com/apache/incubator-doris/pull/4115#discussion_r466895319



##########
File path: be/src/util/metrics.h
##########
@@ -193,230 +174,163 @@ class CoreLocalCounter : public Metric {
     void increment(const T& delta) {
         __sync_fetch_and_add(_value.access(), delta);
     }
+
+    rj::Value to_json_value() const override {
+        return rj::Value(value());
+    }
+
 protected:
     CoreLocalValue<T> _value;
 };
 
 template<typename T>
 class AtomicCounter : public AtomicMetric<T> {
 public:
-    AtomicCounter(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~AtomicCounter() { }
+    AtomicCounter() {}
+    virtual ~AtomicCounter() {}
 };
 
 template<typename T>
 class AtomicGauge : public AtomicMetric<T> {
 public:
-    AtomicGauge(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~AtomicGauge() { }
+    AtomicGauge() : AtomicMetric<T>() {}
+    virtual ~AtomicGauge() {}
 };
 
 template<typename T>
 class LockCounter : public LockSimpleMetric<T> {
 public:
-    LockCounter(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~LockCounter() { }
+    LockCounter() : LockSimpleMetric<T>() {}
+    virtual ~LockCounter() {}
 };
 
 // This can only used for trival type
 template<typename T>
 class LockGauge : public LockSimpleMetric<T> {
 public:
-    LockGauge(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~LockGauge() { }
+    LockGauge() : LockSimpleMetric<T>() {}
+    virtual ~LockGauge() {}
 };
 
-// one key-value pair used to
-struct MetricLabel {
+using Labels = std::unordered_map<std::string, std::string>;
+struct MetricPrototype {
+public:
+    MetricPrototype(MetricType type_,
+                    MetricUnit unit_,
+                    std::string name_,
+                    std::string description_ = "",
+                    std::string group_name_ = "",
+                    Labels labels_ = Labels(),
+                    bool is_core_metric_ = false)
+        : is_core_metric(is_core_metric_),
+          type(type_),
+          unit(unit_),
+          name(std::move(name_)),
+          description(std::move(description_)),
+          group_name(std::move(group_name_)),
+          labels(std::move(labels_)) {}
+
+    std::string simple_name() const;
+    std::string combine_name(const std::string& registry_name) const;
+
+    bool is_core_metric;
+    MetricType type;
+    MetricUnit unit;
     std::string name;
-    std::string value;
+    std::string description;
+    std::string group_name;
+    Labels labels;
+};
 
-    MetricLabel() { }
-    MetricLabel(const std::string& name_, const std::string& value_) :name(name_), value(value_) {
-    }
+#define DEFINE_METRIC(name, type, unit, desc, group, labels, core)      \

Review comment:
       may be used DEFINE_METRIC_PROTOTYPE better

##########
File path: be/src/util/metrics.h
##########
@@ -193,230 +174,163 @@ class CoreLocalCounter : public Metric {
     void increment(const T& delta) {
         __sync_fetch_and_add(_value.access(), delta);
     }
+
+    rj::Value to_json_value() const override {
+        return rj::Value(value());
+    }
+
 protected:
     CoreLocalValue<T> _value;
 };
 
 template<typename T>
 class AtomicCounter : public AtomicMetric<T> {
 public:
-    AtomicCounter(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~AtomicCounter() { }
+    AtomicCounter() {}
+    virtual ~AtomicCounter() {}
 };
 
 template<typename T>
 class AtomicGauge : public AtomicMetric<T> {
 public:
-    AtomicGauge(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~AtomicGauge() { }
+    AtomicGauge() : AtomicMetric<T>() {}
+    virtual ~AtomicGauge() {}
 };
 
 template<typename T>
 class LockCounter : public LockSimpleMetric<T> {
 public:
-    LockCounter(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~LockCounter() { }
+    LockCounter() : LockSimpleMetric<T>() {}
+    virtual ~LockCounter() {}
 };
 
 // This can only used for trival type
 template<typename T>
 class LockGauge : public LockSimpleMetric<T> {
 public:
-    LockGauge(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~LockGauge() { }
+    LockGauge() : LockSimpleMetric<T>() {}
+    virtual ~LockGauge() {}
 };
 
-// one key-value pair used to
-struct MetricLabel {
+using Labels = std::unordered_map<std::string, std::string>;
+struct MetricPrototype {

Review comment:
       What's the purpose of MetricPrototype?
   Under which circumstance, MetricPrototype can be used to split the Metric.

##########
File path: be/src/util/metrics.h
##########
@@ -193,230 +174,163 @@ class CoreLocalCounter : public Metric {
     void increment(const T& delta) {
         __sync_fetch_and_add(_value.access(), delta);
     }
+
+    rj::Value to_json_value() const override {
+        return rj::Value(value());
+    }
+
 protected:
     CoreLocalValue<T> _value;
 };
 
 template<typename T>
 class AtomicCounter : public AtomicMetric<T> {
 public:
-    AtomicCounter(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~AtomicCounter() { }
+    AtomicCounter() {}
+    virtual ~AtomicCounter() {}
 };
 
 template<typename T>
 class AtomicGauge : public AtomicMetric<T> {
 public:
-    AtomicGauge(MetricUnit unit)
-            : AtomicMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~AtomicGauge() { }
+    AtomicGauge() : AtomicMetric<T>() {}
+    virtual ~AtomicGauge() {}
 };
 
 template<typename T>
 class LockCounter : public LockSimpleMetric<T> {
 public:
-    LockCounter(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::COUNTER, unit) {}
-    virtual ~LockCounter() { }
+    LockCounter() : LockSimpleMetric<T>() {}
+    virtual ~LockCounter() {}
 };
 
 // This can only used for trival type
 template<typename T>
 class LockGauge : public LockSimpleMetric<T> {
 public:
-    LockGauge(MetricUnit unit)
-      : LockSimpleMetric<T>(MetricType::GAUGE, unit) {}
-    virtual ~LockGauge() { }
+    LockGauge() : LockSimpleMetric<T>() {}
+    virtual ~LockGauge() {}
 };
 
-// one key-value pair used to
-struct MetricLabel {
+using Labels = std::unordered_map<std::string, std::string>;
+struct MetricPrototype {
+public:
+    MetricPrototype(MetricType type_,
+                    MetricUnit unit_,
+                    std::string name_,
+                    std::string description_ = "",
+                    std::string group_name_ = "",
+                    Labels labels_ = Labels(),
+                    bool is_core_metric_ = false)
+        : is_core_metric(is_core_metric_),
+          type(type_),
+          unit(unit_),
+          name(std::move(name_)),
+          description(std::move(description_)),
+          group_name(std::move(group_name_)),
+          labels(std::move(labels_)) {}
+
+    std::string simple_name() const;
+    std::string combine_name(const std::string& registry_name) const;
+
+    bool is_core_metric;
+    MetricType type;
+    MetricUnit unit;
     std::string name;
-    std::string value;
+    std::string description;
+    std::string group_name;
+    Labels labels;
+};
 
-    MetricLabel() { }
-    MetricLabel(const std::string& name_, const std::string& value_) :name(name_), value(value_) {
-    }
+#define DEFINE_METRIC(name, type, unit, desc, group, labels, core)      \
+    ::doris::MetricPrototype METRIC_##name(type, unit, #name, desc, group, labels, core)
 
-    bool operator==(const MetricLabel& other) const {
-        return name == other.name && value == other.value;
-    }
-    bool operator!=(const MetricLabel& other) const {
-        return !(*this == other);
-    }
-    bool operator<(const MetricLabel& other) const {
-        auto res = name.compare(other.name);
-        if (res == 0) {
-            return value < other.value;
-        }
-        return res < 0;
-    }
-    int compare(const MetricLabel& other) const {
-        auto res = name.compare(other.name);
-        if (res == 0) {
-            return value.compare(other.value);
-        }
-        return res;
-    }
-    std::string to_string() const {
-        return name + "=" + value;
-    }
-};
+#define DEFINE_COUNTER_METRIC_2ARG(name, unit)                          \
+    DEFINE_METRIC(name, MetricType::COUNTER, unit, "", "", Labels(), false)
 
-struct MetricLabels {
-    static MetricLabels EmptyLabels;
-    // used std::set to sort MetricLabel so that we can get compare two MetricLabels
-    std::set<MetricLabel> labels;
+#define DEFINE_COUNTER_METRIC_3ARG(name, unit, desc)                    \
+    DEFINE_METRIC(name, MetricType::COUNTER, unit, desc, "", Labels(), false)
 
-    MetricLabels& add(const std::string& name, const std::string& value) {
-        labels.emplace(name, value);
-        return *this;
-    }
+#define DEFINE_COUNTER_METRIC_5ARG(name, unit, desc, group, labels)     \
+    DEFINE_METRIC(name, MetricType::COUNTER, unit, desc, #group, labels, false)
 
-    bool operator==(const MetricLabels& other) const {
-        if (labels.size() != other.labels.size()) {
-            return false;
-        }
-        auto it = labels.begin();
-        auto other_it = other.labels.begin();
-        while (it != labels.end()) {
-            if (*it != *other_it) {
-                return false;
-            }
-            ++it;
-            ++other_it;
-        }
-        return true;
-    }
-    bool operator<(const MetricLabels& other) const {
-        auto it = labels.begin();
-        auto other_it = other.labels.begin();
-        while (it != labels.end() && other_it != other.labels.end()) {
-            auto res = it->compare(*other_it);
-            if (res < 0) {
-                return true;
-            } else if (res > 0) {
-                return false;
-            }
-            ++it;
-            ++other_it;
-        }
-        if (it == labels.end()) {
-            if (other_it == other.labels.end()) {
-                return false;
-            }
-            return true;
-        } else {
-            return false;
-        }
-    }
-    bool empty() const {
-        return labels.empty();
+#define DEFINE_GAUGE_METRIC_2ARG(name, unit)                            \
+    DEFINE_METRIC(name, MetricType::GAUGE, unit, "", "", Labels(), false)
+
+#define DEFINE_CORE_GAUGE_METRIC_2ARG(name, unit)                       \
+    DEFINE_METRIC(name, MetricType::GAUGE, unit, "", "", Labels(), true)
+
+#define DEFINE_GAUGE_METRIC_3ARG(name, unit, desc)                      \
+    DEFINE_METRIC(name, MetricType::GAUGE, unit, desc, "", Labels(), false)
+
+#define METRIC_REGISTER(entity, metric)                                 \
+    entity->register_metric(&METRIC_##metric, &metric)
+
+#define METRIC_DEREGISTER(entity, metric)                               \
+    entity->deregister_metric(&METRIC_##metric)
+
+// For 'metrics' in MetricEntity.
+struct MetricPrototypeHash {
+    size_t operator()(const MetricPrototype* metric_prototype) const {
+        return std::hash<std::string>()(metric_prototype->group_name.empty() ? metric_prototype->name : metric_prototype->group_name);
     }
+};
 
-    std::string to_string() const {
-        std::stringstream ss;
-        int i = 0;
-        for (auto& label : labels) {
-            if (i++ > 0) {
-                ss << ",";
-            }
-            ss << label.to_string();
-        }
-        return ss.str();
+struct MetricPrototypeEqualTo {
+    bool operator()(const MetricPrototype* first, const MetricPrototype* second) const {
+        return first->group_name == second->group_name && first->name == second->name;
     }
 };
 
-class MetricCollector;
+using MetricByType = std::unordered_map<const MetricPrototype*, Metric*, MetricPrototypeHash, MetricPrototypeEqualTo>;

Review comment:
       MetricMap is better name or not?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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