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

[1/7] kudu git commit: Improved logging of DMS flushes

Repository: kudu
Updated Branches:
  refs/heads/master 471cd1311 -> c1b9f1088


Improved logging of DMS flushes

>From compaction-test.cc's TestRowSetInput,

Before:
I0202 14:52:24.983525 2427953984 delta_tracker.cc:727] T test_tablet_id P a3e9bdf25f0b431ea57d0249535c29b3: Flushing 20 deltas from DMS 0...
I0202 14:52:24.984046 2427953984 delta_tracker.cc:668] T test_tablet_id P a3e9bdf25f0b431ea57d0249535c29b3: Flushed delta block: 0198399763226209 ts range: [11, 30]

After:
I0202 14:45:18.924826 2427953984 delta_tracker.cc:726] T test_tablet_id P 37440651650a4d858267ad408b7e1ac0: Flushing 20 deltas (1759 bytes on disk) from DMS 0
I0202 14:45:18.925750 2427953984 delta_tracker.cc:668] T test_tablet_id P 37440651650a4d858267ad408b7e1ac0: Flushed delta block 0052242700095437 (296 bytes in memory) stats: ts range=[11, 30], delete_count=[0], reinsert_count=[0], update_counts_by_col_id=[12:20,11:20]

Plus one tiny change to clarify logging on flushes.

Change-Id: I8241a6ae33c22838504aa7007463d11167a22e65
Reviewed-on: http://gerrit.cloudera.org:8080/9204
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/adb90a02
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/adb90a02
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/adb90a02

Branch: refs/heads/master
Commit: adb90a02d8ffc7174414608fc71639e8748ca86f
Parents: 471cd13
Author: Will Berkeley <wd...@apache.org>
Authored: Fri Feb 2 14:47:24 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 01:40:36 2018 +0000

----------------------------------------------------------------------
 src/kudu/tablet/delta_tracker.cc | 11 ++++++-----
 src/kudu/tablet/deltafile.h      |  4 ++++
 src/kudu/tablet/tablet.cc        |  2 +-
 3 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/adb90a02/src/kudu/tablet/delta_tracker.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_tracker.cc b/src/kudu/tablet/delta_tracker.cc
index ada9c88..c867bab 100644
--- a/src/kudu/tablet/delta_tracker.cc
+++ b/src/kudu/tablet/delta_tracker.cc
@@ -665,9 +665,9 @@ Status DeltaTracker::FlushDMS(DeltaMemStore* dms,
   gscoped_ptr<DeltaStats> stats;
   RETURN_NOT_OK(dms->FlushToFile(&dfw, &stats));
   RETURN_NOT_OK(dfw.Finish());
-  LOG_WITH_PREFIX(INFO) << "Flushed delta block: " << block_id.ToString()
-                        << " ts range: [" << stats->min_timestamp()
-                        << ", " << stats->max_timestamp() << "]";
+  LOG_WITH_PREFIX(INFO) << "Flushed delta block " << block_id.ToString()
+                        << " (" << dfw.written_size() << " bytes on disk) "
+                        << "stats: " << stats->ToString();
 
   // Now re-open for read
   unique_ptr<ReadableBlock> readable_block;
@@ -678,7 +678,7 @@ Status DeltaTracker::FlushDMS(DeltaMemStore* dms,
                                             REDO,
                                             std::move(options),
                                             dfr));
-  LOG_WITH_PREFIX(INFO) << "Reopened delta block for read: " << block_id.ToString();
+  VLOG_WITH_PREFIX(1) << "Opened new delta block " << block_id.ToString() << " for read";
 
   RETURN_NOT_OK(rowset_metadata_->CommitRedoDeltaDataBlock(dms->id(), block_id));
   if (flush_type == FLUSH_METADATA) {
@@ -724,7 +724,8 @@ Status DeltaTracker::Flush(MetadataFlushType flush_type) {
     redo_delta_stores_.push_back(old_dms);
   }
 
-  LOG_WITH_PREFIX(INFO) << "Flushing " << count << " deltas from DMS " << old_dms->id() << "...";
+  LOG_WITH_PREFIX(INFO) << "Flushing " << count << " deltas (" << old_dms->EstimateSize()
+                        << " bytes in memory) from DMS " << old_dms->id();
 
   // Now, actually flush the contents of the old DMS.
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/adb90a02/src/kudu/tablet/deltafile.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltafile.h b/src/kudu/tablet/deltafile.h
index 8a2d703..d63af84 100644
--- a/src/kudu/tablet/deltafile.h
+++ b/src/kudu/tablet/deltafile.h
@@ -108,6 +108,10 @@ class DeltaFileWriter {
 
   void WriteDeltaStats(const DeltaStats& stats);
 
+  size_t written_size() const {
+    return writer_->written_size();
+  }
+
  private:
   Status DoAppendDelta(const DeltaKey &key, const RowChangeList &delta);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/adb90a02/src/kudu/tablet/tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 77413bb..7c7298c 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1662,7 +1662,7 @@ Status Tablet::Compact(CompactFlags flags) {
   RETURN_NOT_OK_PREPEND(PickRowSetsToCompact(&input, flags),
                         "Failed to pick rowsets to compact");
   LOG_WITH_PREFIX(INFO) << "Compaction: stage 1 complete, picked "
-                        << input.num_rowsets() << " rowsets to compact";
+                        << input.num_rowsets() << " rowsets to compact or flush";
   if (compaction_hooks_) {
     RETURN_NOT_OK_PREPEND(compaction_hooks_->PostSelectIterators(),
                           "PostSelectIterators hook failed");


[4/7] kudu git commit: KUDU-2279 (part 4): enable metrics log by default

Posted by to...@apache.org.
KUDU-2279 (part 4): enable metrics log by default

Change-Id: Iaeb4c546c2800d8f4288a3626b9e998cd7e35e5c
Reviewed-on: http://gerrit.cloudera.org:8080/9178
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/2572c4fa
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/2572c4fa
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/2572c4fa

Branch: refs/heads/master
Commit: 2572c4fa92965f2478415e6c98f250381f69abf4
Parents: a29704e
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jan 31 19:26:57 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 07:58:52 2018 +0000

----------------------------------------------------------------------
 src/kudu/server/server_base_options.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/2572c4fa/src/kudu/server/server_base_options.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base_options.cc b/src/kudu/server/server_base_options.cc
index 06a0c93..d3e8b6b 100644
--- a/src/kudu/server/server_base_options.cc
+++ b/src/kudu/server/server_base_options.cc
@@ -34,7 +34,7 @@ DEFINE_string(server_dump_info_format, "json",
 TAG_FLAG(server_dump_info_path, hidden);
 TAG_FLAG(server_dump_info_format, hidden);
 
-DEFINE_int32(metrics_log_interval_ms, 0,
+DEFINE_int32(metrics_log_interval_ms, 60000,
              "Interval (in milliseconds) at which the server will dump its "
              "metrics to a local log file. The log files are located in the same "
              "directory as specified by the -log_dir flag. If this is not a positive "


[2/7] kudu git commit: KUDU-2279 (part 2): metrics: only emit changed metrics in metrics log

Posted by to...@apache.org.
KUDU-2279 (part 2): metrics: only emit changed metrics in metrics log

This adds a global 'metrics epoch' which can be externally incremented.
When metrics are modified, they remember the epoch of their most recent
modification.

When we dump metrics, we can pass a lower bound in order to see only
metrics which have been modified in or after a given epoch.

This patch updates the metrics logging to only emit metrics that have
changed in each successive line. This should substantially reduce the
size and CPU cost of metric logging on servers with thousands of
tablets.

Change-Id: Ia26be99a1fa96d52e2ca0905844d56c096d3778e
Reviewed-on: http://gerrit.cloudera.org:8080/9176
Tested-by: Kudu Jenkins
Reviewed-by: Will Berkeley <wd...@gmail.com>
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: ab5387403e42df3161492d6f48b9b51bc85c7499
Parents: adb90a0
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jan 31 18:19:51 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 07:58:18 2018 +0000

----------------------------------------------------------------------
 src/kudu/server/server_base.cc | 18 +++++++----
 src/kudu/util/metrics-test.cc  | 61 ++++++++++++++++++++++++++++---------
 src/kudu/util/metrics.cc       | 32 ++++++++++++++++---
 src/kudu/util/metrics.h        | 56 +++++++++++++++++++++++++++++++++-
 4 files changed, 141 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/ab538740/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index ebef92c..d094318 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -613,6 +613,8 @@ void ServerBase::MetricsLoggingThread() {
   // logging metrics.
   const MonoDelta kWaitBetweenFailures = MonoDelta::FromSeconds(60);
 
+  MetricJsonOptions opts;
+  opts.include_raw_histograms = true;
 
   MonoTime next_log = MonoTime::Now();
   while (!stop_background_threads_latch_.WaitUntil(next_log)) {
@@ -623,13 +625,10 @@ void ServerBase::MetricsLoggingThread() {
     buf << "metrics " << GetCurrentTimeMicros() << " ";
 
     // Collect the metrics JSON string.
-    vector<string> metrics;
-    metrics.emplace_back("*");
-    MetricJsonOptions opts;
-    opts.include_raw_histograms = true;
-
+    int64_t this_log_epoch = Metric::current_epoch();
+    Metric::IncrementEpoch();
     JsonWriter writer(&buf, JsonWriter::COMPACT);
-    Status s = metric_registry_->WriteAsJson(&writer, metrics, opts);
+    Status s = metric_registry_->WriteAsJson(&writer, {"*"}, opts);
     if (!s.ok()) {
       WARN_NOT_OK(s, "Unable to collect metrics to log");
       next_log += kWaitBetweenFailures;
@@ -644,6 +643,13 @@ void ServerBase::MetricsLoggingThread() {
       next_log += kWaitBetweenFailures;
       continue;
     }
+
+    // Next time we fetch, only show those that changed after the epoch
+    // we just logged.
+    //
+    // NOTE: we only bump this in the successful log case so that if we failed to
+    // write above, we wouldn't skip any changes.
+    opts.only_modified_in_or_after_epoch = this_log_epoch + 1;
   }
 
   WARN_NOT_OK(log.Close(), "Unable to close metric log");

http://git-wip-us.apache.org/repos/asf/kudu/blob/ab538740/src/kudu/util/metrics-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics-test.cc b/src/kudu/util/metrics-test.cc
index 698fd7d..3d2d374 100644
--- a/src/kudu/util/metrics-test.cc
+++ b/src/kudu/util/metrics-test.cc
@@ -36,6 +36,7 @@
 #include "kudu/util/jsonwriter.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
@@ -62,13 +63,13 @@ class MetricsTest : public KuduTest {
   scoped_refptr<MetricEntity> entity_;
 };
 
-METRIC_DEFINE_counter(test_entity, reqs_pending, "Requests Pending", MetricUnit::kRequests,
-                      "Number of requests pending");
+METRIC_DEFINE_counter(test_entity, test_counter, "My Test Counter", MetricUnit::kRequests,
+                      "Description of test counter");
 
 TEST_F(MetricsTest, SimpleCounterTest) {
   scoped_refptr<Counter> requests =
-    new Counter(&METRIC_reqs_pending);
-  ASSERT_EQ("Number of requests pending", requests->prototype()->description());
+    new Counter(&METRIC_test_counter);
+  ASSERT_EQ("Description of test counter", requests->prototype()->description());
   ASSERT_EQ(0, requests->value());
   requests->Increment();
   ASSERT_EQ(1, requests->value());
@@ -76,13 +77,13 @@ TEST_F(MetricsTest, SimpleCounterTest) {
   ASSERT_EQ(3, requests->value());
 }
 
-METRIC_DEFINE_gauge_uint64(test_entity, fake_memory_usage, "Memory Usage",
-                           MetricUnit::kBytes, "Test Gauge 1");
+METRIC_DEFINE_gauge_uint64(test_entity, test_gauge, "Test uint64 Gauge",
+                           MetricUnit::kBytes, "Description of Test Gauge");
 
 TEST_F(MetricsTest, SimpleAtomicGaugeTest) {
   scoped_refptr<AtomicGauge<uint64_t> > mem_usage =
-    METRIC_fake_memory_usage.Instantiate(entity_, 0);
-  ASSERT_EQ(METRIC_fake_memory_usage.description(), mem_usage->prototype()->description());
+    METRIC_test_gauge.Instantiate(entity_, 0);
+  ASSERT_EQ(METRIC_test_gauge.description(), mem_usage->prototype()->description());
   ASSERT_EQ(0, mem_usage->value());
   mem_usage->IncrementBy(7);
   ASSERT_EQ(7, mem_usage->value());
@@ -90,8 +91,8 @@ TEST_F(MetricsTest, SimpleAtomicGaugeTest) {
   ASSERT_EQ(5, mem_usage->value());
 }
 
-METRIC_DEFINE_gauge_int64(test_entity, test_func_gauge, "Test Gauge", MetricUnit::kBytes,
-                          "Test Gauge 2");
+METRIC_DEFINE_gauge_int64(test_entity, test_func_gauge, "Test Function Gauge",
+                          MetricUnit::kBytes, "Test Gauge 2");
 
 static int64_t MyFunction(int* metric_val) {
   return (*metric_val)++;
@@ -176,8 +177,8 @@ TEST_F(MetricsTest, SimpleHistogramTest) {
 }
 
 TEST_F(MetricsTest, JsonPrintTest) {
-  scoped_refptr<Counter> bytes_seen = METRIC_reqs_pending.Instantiate(entity_);
-  bytes_seen->Increment();
+  scoped_refptr<Counter> test_counter = METRIC_test_counter.Instantiate(entity_);
+  test_counter->Increment();
   entity_->SetAttribute("test_attr", "attr_val");
 
   // Generate the JSON.
@@ -194,7 +195,7 @@ TEST_F(MetricsTest, JsonPrintTest) {
   ASSERT_EQ(1, metrics.size());
   string metric_name;
   ASSERT_OK(reader.ExtractString(metrics[0], "name", &metric_name));
-  ASSERT_EQ("reqs_pending", metric_name);
+  ASSERT_EQ("test_counter", metric_name);
   int64_t metric_value;
   ASSERT_OK(reader.ExtractInt64(metrics[0], "value", &metric_value));
   ASSERT_EQ(1L, metric_value);
@@ -216,7 +217,7 @@ TEST_F(MetricsTest, RetirementTest) {
   FLAGS_metrics_retirement_age_ms = 100;
 
   const string kMetricName = "foo";
-  scoped_refptr<Counter> counter = METRIC_reqs_pending.Instantiate(entity_);
+  scoped_refptr<Counter> counter = METRIC_test_counter.Instantiate(entity_);
   ASSERT_EQ(1, entity_->UnsafeMetricsMapForTests().size());
 
   // Since we hold a reference to the counter, it should not get retired.
@@ -287,7 +288,7 @@ TEST_F(MetricsTest, TestDumpJsonPrototypes) {
   const char* expected =
     "        {\n"
     "            \"name\": \"test_func_gauge\",\n"
-    "            \"label\": \"Test Gauge\",\n"
+    "            \"label\": \"Test Function Gauge\",\n"
     "            \"type\": \"gauge\",\n"
     "            \"unit\": \"bytes\",\n"
     "            \"description\": \"Test Gauge 2\",\n"
@@ -316,4 +317,34 @@ TEST_F(MetricsTest, TestDumpJsonPrototypes) {
   ASSERT_TRUE(ContainsKey(seen_metrics, "test_hist"));
 }
 
+TEST_F(MetricsTest, TestDumpOnlyChanged) {
+  auto GetJson = [&](int64_t since_epoch) {
+    MetricJsonOptions opts;
+    opts.only_modified_in_or_after_epoch = since_epoch;
+    std::ostringstream out;
+    JsonWriter writer(&out, JsonWriter::COMPACT);
+    CHECK_OK(entity_->WriteAsJson(&writer, { "*" }, opts));
+    return out.str();
+  };
+
+  scoped_refptr<Counter> test_counter = METRIC_test_counter.Instantiate(entity_);
+
+  int64_t epoch_when_modified = Metric::current_epoch();
+  test_counter->Increment();
+
+  // If we pass a "since dirty" epoch from before we incremented it, we should
+  // see the metric.
+  for (int i = 0; i < 2; i++) {
+    ASSERT_STR_CONTAINS(GetJson(epoch_when_modified), "{\"name\":\"test_counter\",\"value\":1}");
+    Metric::IncrementEpoch();
+  }
+
+  // If we pass a current epoch, we should see that the metric was not modified.
+  int64_t new_epoch = Metric::current_epoch();
+  ASSERT_STR_NOT_CONTAINS(GetJson(new_epoch), "test_counter");
+  // ... until we modify it again.
+  test_counter->Increment();
+  ASSERT_STR_CONTAINS(GetJson(new_epoch), "{\"name\":\"test_counter\",\"value\":2}");
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/ab538740/src/kudu/util/metrics.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.cc b/src/kudu/util/metrics.cc
index 6565d97..db5b8f5 100644
--- a/src/kudu/util/metrics.cc
+++ b/src/kudu/util/metrics.cc
@@ -249,9 +249,10 @@ Status MetricEntity::WriteAsJson(JsonWriter* writer,
   writer->String("metrics");
   writer->StartArray();
   for (OrderedMetricMap::value_type& val : metrics) {
-    WARN_NOT_OK(val.second->WriteAsJson(writer, opts),
-                strings::Substitute("Failed to write $0 as JSON", val.first));
-
+    if (val.second->ModifiedInOrAfterEpoch(opts.only_modified_in_or_after_epoch)) {
+      WARN_NOT_OK(val.second->WriteAsJson(writer, opts),
+                  strings::Substitute("Failed to write $0 as JSON", val.first));
+    }
   }
   writer->EndArray();
 
@@ -493,13 +494,32 @@ scoped_refptr<MetricEntity> MetricRegistry::FindOrCreateEntity(
 //
 // Metric
 //
+
+std::atomic<int64_t> Metric::g_epoch_;
+
 Metric::Metric(const MetricPrototype* prototype)
-  : prototype_(prototype) {
+    : prototype_(prototype),
+      m_epoch_(current_epoch()) {
 }
 
 Metric::~Metric() {
 }
 
+void Metric::IncrementEpoch() {
+  g_epoch_++;
+}
+
+void Metric::UpdateModificationEpochSlowPath() {
+  int64_t new_epoch, old_epoch;
+  // CAS loop to ensure that we never transition a metric's epoch backwards
+  // even if multiple threads race to update it.
+  do {
+    old_epoch = m_epoch_;
+    new_epoch = g_epoch_;
+  } while (old_epoch < new_epoch &&
+           !m_epoch_.compare_exchange_weak(old_epoch, new_epoch));
+}
+
 //
 // Gauge
 //
@@ -531,6 +551,7 @@ std::string StringGauge::value() const {
 }
 
 void StringGauge::set_value(const std::string& value) {
+  UpdateModificationEpoch();
   std::lock_guard<simple_spinlock> l(lock_);
   value_ = value;
 }
@@ -560,6 +581,7 @@ void Counter::Increment() {
 }
 
 void Counter::IncrementBy(int64_t amount) {
+  UpdateModificationEpoch();
   value_.IncrementBy(amount);
 }
 
@@ -609,10 +631,12 @@ Histogram::Histogram(const HistogramPrototype* proto)
 }
 
 void Histogram::Increment(int64_t value) {
+  UpdateModificationEpoch();
   histogram_->Increment(value);
 }
 
 void Histogram::IncrementBy(int64_t value, int64_t amount) {
+  UpdateModificationEpoch();
   histogram_->IncrementBy(value, amount);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/ab538740/src/kudu/util/metrics.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.h b/src/kudu/util/metrics.h
index aa553c9..9e1cac4 100644
--- a/src/kudu/util/metrics.h
+++ b/src/kudu/util/metrics.h
@@ -223,8 +223,10 @@
 //
 /////////////////////////////////////////////////////
 
+#include <atomic>
 #include <cstddef>
 #include <cstdint>
+#include <limits>
 #include <mutex>
 #include <string>
 #include <unordered_map>
@@ -420,6 +422,13 @@ struct MetricJsonOptions {
   // unit, etc).
   // Default: false
   bool include_schema_info;
+
+  // Try to skip any metrics which have not been modified since before
+  // the given epoch. The current epoch can be fetched using
+  // Metric::current_epoch() and incremented using Metric::IncrementEpoch().
+  //
+  // Note that this is an inclusive bound.
+  int64_t only_modified_in_or_after_epoch = 0;
 };
 
 class MetricEntityPrototype {
@@ -555,13 +564,49 @@ class Metric : public RefCountedThreadSafe<Metric> {
 
   const MetricPrototype* prototype() const { return prototype_; }
 
+  // Return true if this metric has changed in or after the given metrics epoch.
+  bool ModifiedInOrAfterEpoch(int64_t epoch) {
+    return m_epoch_ >= epoch;
+  }
+
+  // Return the current epoch for tracking modification of metrics.
+  // This can be passed as 'MetricJsonOptions::only_modified_since_epoch' to
+  // get a diff of metrics between two points in time.
+  static int64_t current_epoch() {
+    return g_epoch_;
+  }
+
+  // Advance to the next epoch for metrics.
+  // This is cheap for the calling thread but causes some extra work on the paths
+  // of hot metric updaters, so should only be done rarely (eg before dumping
+  // metrics).
+  static void IncrementEpoch();
+
  protected:
   explicit Metric(const MetricPrototype* prototype);
   virtual ~Metric();
 
   const MetricPrototype* const prototype_;
 
+  void UpdateModificationEpoch() {
+    // If we have some upper bound, we need to invalidate it. We use a 'test-and-set'
+    // here to avoid contending on writes to this cacheline.
+    if (m_epoch_ < current_epoch()) {
+      // Out-of-line the uncommon case which requires a bit more code.
+      UpdateModificationEpochSlowPath();
+    }
+  }
+
+  // The last metrics epoch in which this metric was modified.
+  // We use epochs instead of timestamps since we can ensure that epochs
+  // only change rarely. Thus this member is read-mostly and doesn't cause
+  // cacheline bouncing between metrics writers. We also don't need to read
+  // the system clock, which is more expensive compared to reading 'g_epoch_'.
+  std::atomic<int64_t> m_epoch_;
+
  private:
+  void UpdateModificationEpochSlowPath();
+
   friend class MetricEntity;
   friend class RefCountedThreadSafe<Metric>;
 
@@ -570,6 +615,9 @@ class Metric : public RefCountedThreadSafe<Metric> {
   // uninitialized.
   MonoTime retire_time_;
 
+  // See 'current_epoch()'.
+  static std::atomic<int64_t> g_epoch_;
+
   DISALLOW_COPY_AND_ASSIGN(Metric);
 };
 
@@ -794,9 +842,11 @@ class AtomicGauge : public Gauge {
     value_.Store(static_cast<int64_t>(value), kMemOrderNoBarrier);
   }
   void Increment() {
+    UpdateModificationEpoch();
     value_.IncrementBy(1, kMemOrderNoBarrier);
   }
   virtual void IncrementBy(int64_t amount) {
+    UpdateModificationEpoch();
     value_.IncrementBy(amount, kMemOrderNoBarrier);
   }
   void Decrement() {
@@ -925,7 +975,11 @@ class FunctionGauge : public Gauge {
   friend class MetricEntity;
 
   FunctionGauge(const GaugePrototype<T>* proto, Callback<T()> function)
-      : Gauge(proto), function_(std::move(function)) {}
+      : Gauge(proto), function_(std::move(function)) {
+    // Override the modification epoch to the maximum, since we don't have any idea
+    // when the bound function changes value.
+    m_epoch_ = std::numeric_limits<decltype(m_epoch_.load())>::max();
+  }
 
   static T Return(T v) {
     return v;


[6/7] kudu git commit: build-support: fix reporting of build configuration

Posted by to...@apache.org.
build-support: fix reporting of build configuration

Our older Jenkins setup for running flaky test builds seemed to use the
deprecated "build from the source root" setup with cmake. Now, we use a
build dir which isn't the same as the source root, which means we no
longer have a CMakeCache.txt in the source root.

This updates the test reporting to look for CMakeCache.txt in the proper
spot.

Change-Id: Ib6c131f46e75d2eb57d7e0ce9d2ad21e2ad9ab80
Reviewed-on: http://gerrit.cloudera.org:8080/9182
Tested-by: Kudu Jenkins
Reviewed-by: Hao Hao <ha...@cloudera.com>
Reviewed-by: Grant Henke <gr...@gmail.com>


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

Branch: refs/heads/master
Commit: 6e5b15770ee223bfd225f384369b80e1b8475fa6
Parents: 7579da2
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Feb 1 11:55:47 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 07:59:26 2018 +0000

----------------------------------------------------------------------
 build-support/report-test.sh | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/6e5b1577/build-support/report-test.sh
----------------------------------------------------------------------
diff --git a/build-support/report-test.sh b/build-support/report-test.sh
index e375c90..41351d2 100755
--- a/build-support/report-test.sh
+++ b/build-support/report-test.sh
@@ -54,11 +54,13 @@ if ! ( cd $ROOT && git diff --quiet .  && git diff --cached --quiet . ) ; then
   REVISION="${REVISION}-dirty"
 fi
 
+BUILD_ROOT=$(dirname $TEST_EXECUTABLE)/..
+
 # Parse out our "build config" - a space-separated list of tags
 # which include the cmake build type as well as the list of configured
 # sanitizers
 
-CMAKECACHE=$ROOT/CMakeCache.txt
+CMAKECACHE=$BUILD_ROOT/CMakeCache.txt
 BUILD_CONFIG=$(grep '^CMAKE_BUILD_TYPE:' $CMAKECACHE | cut -f 2 -d=)
 if grep -q "KUDU_USE_ASAN:UNINITIALIZED=1" $CMAKECACHE ; then
   BUILD_CONFIG="$BUILD_CONFIG asan"


[7/7] kudu git commit: [thirdparty] fix sized-deallocation issue on OS X 10.11

Posted by to...@apache.org.
[thirdparty] fix sized-deallocation issue on OS X 10.11

Fixed the issue of building gperftools 2.6.3 on OS X 10.11.  Also,
updated the top-level CMakeLists.txt to enable or disable the sized
deallocation feature for Kudu binaries accordingly.

Change-Id: I12632df70137bf5aed8b44d613b08856a925d840
Reviewed-on: http://gerrit.cloudera.org:8080/9159
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/c1b9f108
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/c1b9f108
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/c1b9f108

Branch: refs/heads/master
Commit: c1b9f1088f830235f9d4e6830f59b1a200a65ab6
Parents: 6e5b157
Author: Alexey Serbin <as...@cloudera.com>
Authored: Mon Jan 29 18:56:35 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 08:01:09 2018 +0000

----------------------------------------------------------------------
 CMakeLists.txt                                  | 17 +++++----
 thirdparty/download-thirdparty.sh               |  3 +-
 .../gperftools-sized-alloc-build-fix.patch      | 36 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c1b9f108/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index e60170a..9e0ca1e 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1084,14 +1084,19 @@ ADD_THIRDPARTY_LIB(boost_date_time
 # compiler options since we only do this when tcmalloc was
 # found and enabled.
 ############################################################
-if (KUDU_TCMALLOC_AVAILABLE AND
-    (("${COMPILER_FAMILY}" STREQUAL "clang" AND
-      "${COMPILER_VERSION}" VERSION_GREATER "3.7") OR
-     ("${COMPILER_FAMILY}" STREQUAL "gcc" AND
-      "${COMPILER_VERSION}" VERSION_GREATER "5.0")))
+execute_process(
+  COMMAND printf
+    "#include <new>\nint main(){(::operator delete)(0,256);return 0;}\n"
+  COMMAND ${CMAKE_CXX_COMPILER}
+    -x c++ -fsized-deallocation -O0 -fno-builtin
+    -l:${TCMALLOC_SHARED_LIB} -o /dev/null -
+  RESULT_VARIABLE SIZED_DEALLOCATION_TEST_RESULT OUTPUT_QUIET ERROR_QUIET)
+if (${SIZED_DEALLOCATION_TEST_RESULT} EQUAL 0)
   set(COMPILER_SUPPORTS_SIZED_DEALLOCATION TRUE)
   set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation")
-  message("Enabling sized-deallocation")
+  message("sized-deallocation is ENABLED")
+else()
+  message("sized-deallocation is DISABLED")
 endif()
 
 ############################################################

http://git-wip-us.apache.org/repos/asf/kudu/blob/c1b9f108/thirdparty/download-thirdparty.sh
----------------------------------------------------------------------
diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh
index d699429..3bff35b 100755
--- a/thirdparty/download-thirdparty.sh
+++ b/thirdparty/download-thirdparty.sh
@@ -124,13 +124,14 @@ if [ ! -d $GFLAGS_SOURCE ]; then
   fetch_and_expand gflags-${GFLAGS_VERSION}.tar.gz
 fi
 
-GPERFTOOLS_PATCHLEVEL=1
+GPERFTOOLS_PATCHLEVEL=2
 delete_if_wrong_patchlevel $GPERFTOOLS_SOURCE $GPERFTOOLS_PATCHLEVEL
 if [ ! -d $GPERFTOOLS_SOURCE ]; then
   fetch_and_expand gperftools-${GPERFTOOLS_VERSION}.tar.gz
 
   pushd $GPERFTOOLS_SOURCE
   patch -p1 < $TP_DIR/patches/gperftools-Replace-namespace-base-with-namespace-tcmalloc.patch
+  patch -p1 < $TP_DIR/patches/gperftools-sized-alloc-build-fix.patch
   touch patchlevel-$GPERFTOOLS_PATCHLEVEL
   autoreconf -fvi
   popd

http://git-wip-us.apache.org/repos/asf/kudu/blob/c1b9f108/thirdparty/patches/gperftools-sized-alloc-build-fix.patch
----------------------------------------------------------------------
diff --git a/thirdparty/patches/gperftools-sized-alloc-build-fix.patch b/thirdparty/patches/gperftools-sized-alloc-build-fix.patch
new file mode 100644
index 0000000..2566070
--- /dev/null
+++ b/thirdparty/patches/gperftools-sized-alloc-build-fix.patch
@@ -0,0 +1,36 @@
+--- ./configure.ac.orig	2017-12-09 09:43:20.000000000 -0800
++++ ./configure.ac	2018-01-29 17:31:48.000000000 -0800
+@@ -370,8 +370,8 @@
+                [perftools_cv_sized_deallocation_result],
+                [AC_LANG_PUSH(C++)
+                 OLD_CXXFLAGS="$CXXFLAGS"
+-                CXXFLAGS="$CXXFLAGS -fsized-deallocation"
+-                AC_COMPILE_IFELSE([AC_LANG_PROGRAM(
++                CXXFLAGS="$CXXFLAGS -fsized-deallocation -O0 -fno-builtin"
++                AC_LINK_IFELSE([AC_LANG_PROGRAM(
+                     [[#include <new>]],
+                     [[(::operator delete)(0, 256)]])],
+                  perftools_cv_sized_deallocation_result=yes,
+@@ -385,18 +385,21 @@
+ AC_CACHE_CHECK([if C++ compiler supports std::align_val_t without options],
+                [perftools_cv_have_align_val_t],
+                [AC_LANG_PUSH(C++)
++                OLD_CXXFLAGS="$CXXFLAGS"
++                CXXFLAGS="$CXXFLAGS -O0 -fno-builtin"
+                 AC_LINK_IFELSE([AC_LANG_PROGRAM(
+                     [[#include <new>]],
+                     [[(::operator delete)((::operator new)(256, std::align_val_t(16)), std::align_val_t(16))]])],
+                  perftools_cv_have_align_val_t=yes,
+                  perftools_cv_have_align_val_t=no)
++                CXXFLAGS="$OLD_CXXFLAGS"
+                 AC_LANG_POP(C++)])
+ 
+ AC_CACHE_CHECK([if C++ compiler supports -faligned-new],
+                [perftools_cv_have_f_aligned_new],
+                [AC_LANG_PUSH(C++)
+                 OLD_CXXFLAGS="$CXXFLAGS"
+-                CXXFLAGS="$CXXFLAGS -faligned-new"
++                CXXFLAGS="$CXXFLAGS -faligned-new -O0 -fno-builtin"
+                 AC_LINK_IFELSE([AC_LANG_PROGRAM(
+                     [[#include <new>]],
+                     [[(::operator delete)((::operator new)(256, std::align_val_t(16)), std::align_val_t(16))]])],


[5/7] kudu git commit: KUDU-2279 (part 5): don't include entity attributes in log

Posted by to...@apache.org.
KUDU-2279 (part 5): don't include entity attributes in log

Entity attributes like table names and partition descriptions aren't
very useful in the context of the metrics log. They can always be
grabbed from other places like 'kudu remote_replica list', etc, and are
rarely relevant for tserver-scope debugging.

Additionally, these might be considered somewhat sensitive for users to
share publically in bug reports, etc, so may as well "redact" them.

This shaves a few more bytes off the entries in the metrics log.

Change-Id: Ic995f6a3b8430b73a041de8e5bcbc53a5d8e7f1b
Reviewed-on: http://gerrit.cloudera.org:8080/9180
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/7579da22
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/7579da22
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/7579da22

Branch: refs/heads/master
Commit: 7579da22adb758a680b2ab09d94a86a92b4e18c5
Parents: 2572c4f
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jan 31 19:49:10 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 6 07:59:14 2018 +0000

----------------------------------------------------------------------
 src/kudu/server/server_base.cc |  4 ++++
 src/kudu/util/metrics.cc       | 14 ++++++++------
 src/kudu/util/metrics.h        |  3 +++
 3 files changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/7579da22/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index e5a5a9a..95d3e65 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -621,6 +621,10 @@ void ServerBase::MetricsLoggingThread() {
   // also ensures that we don't dump a bunch of zero data on startup.
   opts.include_untouched_metrics = false;
 
+  // Entity attributes aren't that useful in the context of this log. We can
+  // always grab the entity attributes separately if necessary.
+  opts.include_entity_attributes = 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/7579da22/src/kudu/util/metrics.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.cc b/src/kudu/util/metrics.cc
index 911cab6..218b08e 100644
--- a/src/kudu/util/metrics.cc
+++ b/src/kudu/util/metrics.cc
@@ -238,13 +238,15 @@ Status MetricEntity::WriteAsJson(JsonWriter* writer,
   writer->String("id");
   writer->String(id_);
 
-  writer->String("attributes");
-  writer->StartObject();
-  for (const AttributeMap::value_type& val : attrs) {
-    writer->String(val.first);
-    writer->String(val.second);
+  if (opts.include_entity_attributes) {
+    writer->String("attributes");
+    writer->StartObject();
+    for (const AttributeMap::value_type& val : attrs) {
+      writer->String(val.first);
+      writer->String(val.second);
+    }
+    writer->EndObject();
   }
-  writer->EndObject();
 
   writer->String("metrics");
   writer->StartArray();

http://git-wip-us.apache.org/repos/asf/kudu/blob/7579da22/src/kudu/util/metrics.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/metrics.h b/src/kudu/util/metrics.h
index 3876405..65ad030 100644
--- a/src/kudu/util/metrics.h
+++ b/src/kudu/util/metrics.h
@@ -435,6 +435,9 @@ struct MetricJsonOptions {
   // 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;
+
+  // Whether to include the attributes of each entity.
+  bool include_entity_attributes = true;
 };
 
 class MetricEntityPrototype {


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

Posted by to...@apache.org.
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;