You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2018/02/06 08:08:54 UTC

[3/7] kudu git commit: KUDU-2279 (part 3): metrics: don't emit untouched metrics to log

KUDU-2279 (part 3): metrics: don't emit untouched metrics to log

This avoids a big dump of zero-valued metrics at startup.

Change-Id: I92d2c4640e54c91791fab9c420215bafa3fe8f20
Reviewed-on: http://gerrit.cloudera.org:8080/9177
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/a29704e2
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/a29704e2
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/a29704e2

Branch: refs/heads/master
Commit: a29704e2067fc28c6d0b54b8ae5a28102b788f4f
Parents: ab53874
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jan 31 19:16:47 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 07:58:40 2018 +0000

----------------------------------------------------------------------
 src/kudu/server/server_base.cc |  5 +++++
 src/kudu/util/metrics-test.cc  | 28 ++++++++++++++++++++++++++++
 src/kudu/util/metrics.cc       |  8 ++++++--
 src/kudu/util/metrics.h        | 30 +++++++++++++++++++++++++++++-
 4 files changed, 68 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a29704e2/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index d094318..e5a5a9a 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -616,6 +616,11 @@ void ServerBase::MetricsLoggingThread() {
   MetricJsonOptions opts;
   opts.include_raw_histograms = true;
 
+  // We don't output any metrics which have never been incremented. Though
+  // this seems redundant with the "only include changed metrics" above, it
+  // also ensures that we don't dump a bunch of zero data on startup.
+  opts.include_untouched_metrics = false;
+
   MonoTime next_log = MonoTime::Now();
   while (!stop_background_threads_latch_.WaitUntil(next_log)) {
     next_log = MonoTime::Now() +

http://git-wip-us.apache.org/repos/asf/kudu/blob/a29704e2/src/kudu/util/metrics-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics-test.cc b/src/kudu/util/metrics-test.cc
index 3d2d374..246397d 100644
--- a/src/kudu/util/metrics-test.cc
+++ b/src/kudu/util/metrics-test.cc
@@ -347,4 +347,32 @@ TEST_F(MetricsTest, TestDumpOnlyChanged) {
   ASSERT_STR_CONTAINS(GetJson(new_epoch), "{\"name\":\"test_counter\",\"value\":2}");
 }
 
+
+// Test that 'include_untouched_metrics=false' prevents dumping counters and histograms
+// which have never been incremented.
+TEST_F(MetricsTest, TestDontDumpUntouched) {
+  // Instantiate a bunch of metrics.
+  int metric_val = 1000;
+  scoped_refptr<Counter> test_counter = METRIC_test_counter.Instantiate(entity_);
+  scoped_refptr<Histogram> hist = METRIC_test_hist.Instantiate(entity_);
+  scoped_refptr<FunctionGauge<int64_t> > function_gauge =
+    METRIC_test_func_gauge.InstantiateFunctionGauge(
+        entity_, Bind(&MyFunction, Unretained(&metric_val)));
+  scoped_refptr<AtomicGauge<uint64_t> > atomic_gauge =
+    METRIC_test_gauge.Instantiate(entity_, 0);
+
+  MetricJsonOptions opts;
+  opts.include_untouched_metrics = false;
+  std::ostringstream out;
+  JsonWriter writer(&out, JsonWriter::COMPACT);
+  CHECK_OK(entity_->WriteAsJson(&writer, { "*" }, opts));
+  // Untouched counters and histograms should not be included.
+  ASSERT_STR_NOT_CONTAINS(out.str(), "test_counter");
+  ASSERT_STR_NOT_CONTAINS(out.str(), "test_hist");
+  // Untouched gauges need to be included, because we don't actually
+  // track whether they have been touched.
+  ASSERT_STR_CONTAINS(out.str(), "test_func_gauge");
+  ASSERT_STR_CONTAINS(out.str(), "test_gauge");
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/a29704e2/src/kudu/util/metrics.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.cc b/src/kudu/util/metrics.cc
index db5b8f5..911cab6 100644
--- a/src/kudu/util/metrics.cc
+++ b/src/kudu/util/metrics.cc
@@ -249,8 +249,12 @@ Status MetricEntity::WriteAsJson(JsonWriter* writer,
   writer->String("metrics");
   writer->StartArray();
   for (OrderedMetricMap::value_type& val : metrics) {
-    if (val.second->ModifiedInOrAfterEpoch(opts.only_modified_in_or_after_epoch)) {
-      WARN_NOT_OK(val.second->WriteAsJson(writer, opts),
+    const auto& m = val.second;
+    if (m->ModifiedInOrAfterEpoch(opts.only_modified_in_or_after_epoch)) {
+      if (!opts.include_untouched_metrics && m->IsUntouched()) {
+        continue;
+      }
+      WARN_NOT_OK(m->WriteAsJson(writer, opts),
                   strings::Substitute("Failed to write $0 as JSON", val.first));
     }
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/a29704e2/src/kudu/util/metrics.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.h b/src/kudu/util/metrics.h
index 9e1cac4..3876405 100644
--- a/src/kudu/util/metrics.h
+++ b/src/kudu/util/metrics.h
@@ -429,6 +429,12 @@ struct MetricJsonOptions {
   //
   // Note that this is an inclusive bound.
   int64_t only_modified_in_or_after_epoch = 0;
+
+  // Whether to include metrics which have had no data recorded and thus have
+  // a value of 0. Note that some metrics with the value 0 may still be included:
+  // notably, gauges may be non-zero and then reset to zero, so seeing that
+  // they are currently zero does not indicate they are "untouched".
+  bool include_untouched_metrics = true;
 };
 
 class MetricEntityPrototype {
@@ -564,6 +570,9 @@ class Metric : public RefCountedThreadSafe<Metric> {
 
   const MetricPrototype* prototype() const { return prototype_; }
 
+  // Return true if this metric has never been touched.
+  virtual bool IsUntouched() const = 0;
+
   // Return true if this metric has changed in or after the given metrics epoch.
   bool ModifiedInOrAfterEpoch(int64_t epoch) {
     return m_epoch_ >= epoch;
@@ -806,6 +815,7 @@ class Gauge : public Metric {
   virtual ~Gauge() {}
   virtual Status WriteAsJson(JsonWriter* w,
                              const MetricJsonOptions& opts) const OVERRIDE;
+
  protected:
   virtual void WriteValue(JsonWriter* writer) const = 0;
  private:
@@ -819,6 +829,10 @@ class StringGauge : public Gauge {
               std::string initial_value);
   std::string value() const;
   void set_value(const std::string& value);
+  virtual bool IsUntouched() const override {
+    return false;
+  }
+
  protected:
   virtual void WriteValue(JsonWriter* writer) const OVERRIDE;
  private:
@@ -855,7 +869,9 @@ class AtomicGauge : public Gauge {
   void DecrementBy(int64_t amount) {
     IncrementBy(-amount);
   }
-
+  virtual bool IsUntouched() const override {
+    return false;
+  }
  protected:
   virtual void WriteValue(JsonWriter* writer) const OVERRIDE {
     writer->Value(value());
@@ -971,6 +987,10 @@ class FunctionGauge : public Gauge {
                                 this));
   }
 
+  virtual bool IsUntouched() const override {
+    return false;
+  }
+
  private:
   friend class MetricEntity;
 
@@ -1018,6 +1038,10 @@ class Counter : public Metric {
   virtual Status WriteAsJson(JsonWriter* w,
                              const MetricJsonOptions& opts) const OVERRIDE;
 
+  virtual bool IsUntouched() const override {
+    return value() == 0;
+  }
+
  private:
   FRIEND_TEST(MetricsTest, SimpleCounterTest);
   FRIEND_TEST(MultiThreadedMetricsTest, CounterIncrementTest);
@@ -1073,6 +1097,10 @@ class Histogram : public Metric {
 
   const HdrHistogram* histogram_for_tests() const { return histogram_.get(); }
 
+  virtual bool IsUntouched() const override {
+    return TotalCount() == 0;
+  }
+
  private:
   FRIEND_TEST(MetricsTest, SimpleHistogramTest);
   friend class MetricEntity;