You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/06/05 04:01:18 UTC

[impala] 02/02: IMPALA-8578: part 2: move metrics code to .cc files

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

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 95a1da2d32ea7b28585ad574b22c2bb9dd921029
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Mon Jun 3 12:31:30 2019 -0700

    IMPALA-8578: part 2: move metrics code to .cc files
    
    This moves a lot of metric function definitions into .cc files,
    to reduce the size of compilation units and to reduce the
    frequency of recompilation required when making changes
    to metrics.
    
    This moves most of the large, non-perf-critical metric
    functions into .cc files. For template classes, this
    requires explicitly instantiating all combinations of
    template parameters that are used in impala, including
    in tests.
    
    Disable weak-template-vtables warning because of
    spurious warnings on template instantiations. See
    https://bugs.llvm.org/show_bug.cgi?id=18733
    
    Change-Id: I78ad045ded6e6a7b7524711be9302c26115b97b9
    Reviewed-on: http://gerrit.cloudera.org:8080/13501
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .clang-tidy                                        |   1 +
 be/src/exprs/expr-test.cc                          |   2 +
 be/src/rpc/rpc-trace.cc                            |   2 +
 be/src/rpc/rpc-trace.h                             |   7 +-
 be/src/rpc/thrift-server.cc                        |   1 +
 .../runtime/bufferpool/reservation-tracker-test.cc |   1 +
 be/src/runtime/coordinator.cc                      |   1 +
 be/src/runtime/io/hdfs-file-reader.cc              |   1 +
 be/src/runtime/krpc-data-stream-mgr.cc             |   3 +-
 be/src/runtime/runtime-filter-bank.cc              |   3 +
 be/src/scheduling/admission-controller-test.cc     |   1 +
 be/src/scheduling/admission-controller.cc          |   1 +
 be/src/scheduling/scheduler.cc                     |   1 +
 be/src/service/client-request-state.cc             |   2 +
 be/src/service/client-request-state.h              |   5 +-
 be/src/service/impala-hs2-server.cc                |   1 +
 be/src/service/impala-http-handler.cc              |   1 +
 be/src/service/impala-server.cc                    |   1 +
 be/src/service/impala-server.h                     |   1 -
 be/src/statestore/statestore.cc                    |   2 +
 be/src/statestore/statestore.h                     |   6 +-
 be/src/util/CMakeLists.txt                         |   2 +
 be/src/util/auth-util.cc                           |   5 +-
 be/src/util/collection-metrics.cc                  | 236 +++++++++++++++++++++
 be/src/util/collection-metrics.h                   | 210 ++----------------
 be/src/util/default-path-handlers.cc               |   1 +
 be/src/util/histogram-metric.cc                    | 177 ++++++++++++++++
 be/src/util/histogram-metric.h                     | 158 +-------------
 be/src/util/metrics-test.cc                        |   4 +-
 be/src/util/metrics.cc                             | 114 +++++++++-
 be/src/util/metrics.h                              | 102 ++-------
 be/src/util/webserver.cc                           |  13 +-
 be/src/util/webserver.h                            |  12 +-
 33 files changed, 620 insertions(+), 458 deletions(-)

diff --git a/.clang-tidy b/.clang-tidy
index bf37d7a..a99d528 100644
--- a/.clang-tidy
+++ b/.clang-tidy
@@ -69,6 +69,7 @@ Checks: "-*,clang*,\
 -clang-diagnostic-unused-parameter,\
 -clang-diagnostic-used-but-marked-unused,\
 -clang-diagnostic-vla-extension,\
+-clang-diagnostic-weak-template-vtables,\
 -clang-diagnostic-weak-vtables"
 
 # Ignore warnings in gutil
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index 4e32ce5..f8486b9 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -21,6 +21,7 @@
 #include <map>
 #include <string>
 
+#include <boost/algorithm/string.hpp>
 #include <boost/date_time/c_local_time_adjustor.hpp>
 #include <boost/date_time/posix_time/posix_time.hpp>
 #include <boost/lexical_cast.hpp>
@@ -64,6 +65,7 @@
 #include "udf/udf-test-harness.h"
 #include "util/asan.h"
 #include "util/debug-util.h"
+#include "util/metrics.h"
 #include "util/string-parser.h"
 #include "util/string-util.h"
 #include "util/test-info.h"
diff --git a/be/src/rpc/rpc-trace.cc b/be/src/rpc/rpc-trace.cc
index 0febbbe..87bfad5 100644
--- a/be/src/rpc/rpc-trace.cc
+++ b/be/src/rpc/rpc-trace.cc
@@ -24,6 +24,8 @@
 #include "common/logging.h"
 #include "rpc/rpc-mgr.h"
 #include "util/debug-util.h"
+#include "util/histogram-metric.h"
+#include "util/pretty-printer.h"
 #include "util/time.h"
 #include "util/webserver.h"
 
diff --git a/be/src/rpc/rpc-trace.h b/be/src/rpc/rpc-trace.h
index 12c7395..0a743c8 100644
--- a/be/src/rpc/rpc-trace.h
+++ b/be/src/rpc/rpc-trace.h
@@ -15,11 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_RPC_RPC_TRACE_H
-#define IMPALA_RPC_RPC_TRACE_H
+#pragma once
 
 #include "rpc/thrift-server.h"
-#include "util/histogram-metric.h"
+#include "util/metrics-fwd.h"
 #include "util/internal-queue.h"
 
 #include <thrift/TProcessor.h>
@@ -130,5 +129,3 @@ class RpcEventHandler : public apache::thrift::TProcessorEventHandler {
 void InitRpcEventTracing(Webserver* webserver, RpcMgr* = nullptr);
 
 }
-
-#endif
diff --git a/be/src/rpc/thrift-server.cc b/be/src/rpc/thrift-server.cc
index aa9a088..84c17cc 100644
--- a/be/src/rpc/thrift-server.cc
+++ b/be/src/rpc/thrift-server.cc
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <boost/algorithm/string.hpp>
 #include <boost/filesystem.hpp>
 #include <boost/thread.hpp>
 #include <boost/thread/mutex.hpp>
diff --git a/be/src/runtime/bufferpool/reservation-tracker-test.cc b/be/src/runtime/bufferpool/reservation-tracker-test.cc
index c03d685..bf74c31 100644
--- a/be/src/runtime/bufferpool/reservation-tracker-test.cc
+++ b/be/src/runtime/bufferpool/reservation-tracker-test.cc
@@ -27,6 +27,7 @@
 #include "common/object-pool.h"
 #include "runtime/mem-tracker.h"
 #include "util/memory-metrics.h"
+#include "util/pretty-printer.h"
 #include "testutil/gtest-util.h"
 
 #include "common/names.h"
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 5d10857..1ef3ca5 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -49,6 +49,7 @@
 #include "util/hdfs-util.h"
 #include "util/histogram-metric.h"
 #include "util/min-max-filter.h"
+#include "util/pretty-printer.h"
 #include "util/table-printer.h"
 
 #include "common/names.h"
diff --git a/be/src/runtime/io/hdfs-file-reader.cc b/be/src/runtime/io/hdfs-file-reader.cc
index 93f1c09..7e959ae 100644
--- a/be/src/runtime/io/hdfs-file-reader.cc
+++ b/be/src/runtime/io/hdfs-file-reader.cc
@@ -26,6 +26,7 @@
 #include "util/hdfs-util.h"
 #include "util/impalad-metrics.h"
 #include "util/metrics.h"
+#include "util/pretty-printer.h"
 #include "util/scope-exit-trigger.h"
 
 #include "common/names.h"
diff --git a/be/src/runtime/krpc-data-stream-mgr.cc b/be/src/runtime/krpc-data-stream-mgr.cc
index 07047c4..521f52c 100644
--- a/be/src/runtime/krpc-data-stream-mgr.cc
+++ b/be/src/runtime/krpc-data-stream-mgr.cc
@@ -27,8 +27,8 @@
 
 #include "exec/kudu-util.h"
 #include "runtime/exec-env.h"
-#include "runtime/mem-tracker.h"
 #include "runtime/krpc-data-stream-recvr.h"
+#include "runtime/mem-tracker.h"
 #include "runtime/raw-value.inline.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
@@ -37,6 +37,7 @@
 #include "util/debug-util.h"
 #include "util/metrics.h"
 #include "util/periodic-counter-updater.h"
+#include "util/pretty-printer.h"
 #include "util/runtime-profile-counters.h"
 #include "util/uid-util.h"
 
diff --git a/be/src/runtime/runtime-filter-bank.cc b/be/src/runtime/runtime-filter-bank.cc
index f8667bc..11026cb 100644
--- a/be/src/runtime/runtime-filter-bank.cc
+++ b/be/src/runtime/runtime-filter-bank.cc
@@ -17,6 +17,8 @@
 
 #include "runtime/runtime-filter-bank.h"
 
+#include <boost/algorithm/string/join.hpp>
+
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "gutil/strings/substitute.h"
 #include "runtime/client-cache.h"
@@ -32,6 +34,7 @@
 #include "util/bit-util.h"
 #include "util/bloom-filter.h"
 #include "util/min-max-filter.h"
+#include "util/pretty-printer.h"
 
 #include "common/names.h"
 
diff --git a/be/src/scheduling/admission-controller-test.cc b/be/src/scheduling/admission-controller-test.cc
index c3fa309..54b47e4 100644
--- a/be/src/scheduling/admission-controller-test.cc
+++ b/be/src/scheduling/admission-controller-test.cc
@@ -25,6 +25,7 @@
 #include "runtime/test-env.h"
 #include "service/fe-support.h"
 #include "testutil/gtest-util.h"
+#include "util/metrics.h"
 
 // Access the flags that are defined in RequestPoolService.
 DECLARE_string(fair_scheduler_allocation_path);
diff --git a/be/src/scheduling/admission-controller.cc b/be/src/scheduling/admission-controller.cc
index 05b3274..5564bb0b 100644
--- a/be/src/scheduling/admission-controller.cc
+++ b/be/src/scheduling/admission-controller.cc
@@ -28,6 +28,7 @@
 #include "scheduling/scheduler.h"
 #include "util/bit-util.h"
 #include "util/debug-util.h"
+#include "util/metrics.h"
 #include "util/pretty-printer.h"
 #include "util/runtime-profile-counters.h"
 #include "util/time.h"
diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc
index 858234b..d4183e3 100644
--- a/be/src/scheduling/scheduler.cc
+++ b/be/src/scheduling/scheduler.cc
@@ -41,6 +41,7 @@
 #include "util/hash-util.h"
 #include "util/metrics.h"
 #include "util/network-util.h"
+#include "util/pretty-printer.h"
 #include "util/runtime-profile-counters.h"
 
 #include "common/names.h"
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index f4dd448..1427fbb 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -17,6 +17,7 @@
 
 #include "service/client-request-state.h"
 
+#include <boost/algorithm/string/join.hpp>
 #include <boost/algorithm/string/predicate.hpp>
 #include <limits>
 #include <gutil/strings/substitute.h>
@@ -39,6 +40,7 @@
 #include "util/debug-util.h"
 #include "util/impalad-metrics.h"
 #include "util/metrics.h"
+#include "util/pretty-printer.h"
 #include "util/runtime-profile-counters.h"
 #include "util/time.h"
 
diff --git a/be/src/service/client-request-state.h b/be/src/service/client-request-state.h
index 1d9042a..ae3dc95 100644
--- a/be/src/service/client-request-state.h
+++ b/be/src/service/client-request-state.h
@@ -15,10 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_SERVICE_CLIENT_REQUEST_STATE_H
-#define IMPALA_SERVICE_CLIENT_REQUEST_STATE_H
+#pragma once
 
 #include "common/atomic.h"
+#include "common/object-pool.h"
 #include "common/status.h"
 #include "exec/catalog-op-executor.h"
 #include "runtime/timestamp-value.h"
@@ -526,4 +526,3 @@ class ClientRequestState {
 };
 
 }
-#endif
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 34a0de4..8b7896b 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -49,6 +49,7 @@
 #include "util/debug-util.h"
 #include "util/impalad-metrics.h"
 #include "util/metrics.h"
+#include "util/pretty-printer.h"
 #include "util/runtime-profile-counters.h"
 #include "util/string-parser.h"
 
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index e9941da..23b86a3 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -40,6 +40,7 @@
 #include "thrift/protocol/TDebugProtocol.h"
 #include "util/coding-util.h"
 #include "util/logging-support.h"
+#include "util/pretty-printer.h"
 #include "util/redactor.h"
 #include "util/summary-util.h"
 #include "util/time.h"
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 8200445..eb6bec7 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -77,6 +77,7 @@
 #include "util/network-util.h"
 #include "util/openssl-util.h"
 #include "util/parse-util.h"
+#include "util/pretty-printer.h"
 #include "util/redactor.h"
 #include "util/runtime-profile-counters.h"
 #include "util/runtime-profile.h"
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index 8f6b7e6..fdbdb7f 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -38,7 +38,6 @@
 #include "service/query-options.h"
 #include "util/condition-variable.h"
 #include "util/container-util.h"
-#include "util/metrics.h"
 #include "util/runtime-profile.h"
 #include "util/sharded-query-map-util.h"
 #include "util/simple-logger.h"
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index d7eb152..7493e3a 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -33,11 +33,13 @@
 #include "rpc/thrift-util.h"
 #include "statestore/failure-detector.h"
 #include "statestore/statestore-subscriber-client-wrapper.h"
+#include "util/collection-metrics.h"
 #include "util/container-util.h"
 #include "util/debug-util.h"
 #include "util/logging-support.h"
 #include "util/metrics.h"
 #include "util/openssl-util.h"
+#include "util/pretty-printer.h"
 #include "util/test-info.h"
 #include "util/time.h"
 #include "util/uid-util.h"
diff --git a/be/src/statestore/statestore.h b/be/src/statestore/statestore.h
index 8bb6d1b..84c6121 100644
--- a/be/src/statestore/statestore.h
+++ b/be/src/statestore/statestore.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef STATESTORE_STATESTORE_H
-#define STATESTORE_STATESTORE_H
+#pragma once
 
 #include <atomic>
 #include <cstdint>
@@ -41,7 +40,6 @@
 #include "statestore/failure-detector.h"
 #include "statestore/statestore-subscriber-client-wrapper.h"
 #include "util/aligned-new.h"
-#include "util/collection-metrics.h"
 #include "util/metrics-fwd.h"
 #include "util/thread-pool.h"
 #include "util/webserver.h"
@@ -723,5 +721,3 @@ class Statestore : public CacheLineAligned {
 };
 
 } // namespace impala
-
-#endif
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 35a53f7..e07ee59 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -38,6 +38,7 @@ add_library(Util
   cgroup-util.cc
   coding-util.cc
   codec.cc
+  collection-metrics.cc
   common-metrics.cc
   compress.cc
   cpu-info.cc
@@ -54,6 +55,7 @@ add_library(Util
   hdfs-util.cc
   hdfs-bulk-ops.cc
   hdr-histogram.cc
+  histogram-metric.cc
   impalad-metrics.cc
   jni-util.cc
   logging-support.cc
diff --git a/be/src/util/auth-util.cc b/be/src/util/auth-util.cc
index e8a0e41..218c211 100644
--- a/be/src/util/auth-util.cc
+++ b/be/src/util/auth-util.cc
@@ -15,8 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "service/client-request-state.h"
 #include "util/auth-util.h"
+
+#include <boost/algorithm/string.hpp>
+
+#include "service/client-request-state.h"
 #include "util/network-util.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 
diff --git a/be/src/util/collection-metrics.cc b/be/src/util/collection-metrics.cc
new file mode 100644
index 0000000..1160ccf
--- /dev/null
+++ b/be/src/util/collection-metrics.cc
@@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/collection-metrics.h"
+
+#include "util/json-util.h"
+#include "util/pretty-printer.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+template <typename T>
+void SetMetric<T>::ToJson(rapidjson::Document* document, rapidjson::Value* value) {
+  rapidjson::Value container(rapidjson::kObjectType);
+  AddStandardFields(document, &container);
+  rapidjson::Value metric_list(rapidjson::kArrayType);
+  for (const T& s: value_) {
+    rapidjson::Value entry_value;
+    ToJsonValue(s, TUnit::NONE, document, &entry_value);
+    metric_list.PushBack(entry_value, document->GetAllocator());
+  }
+  container.AddMember("items", metric_list, document->GetAllocator());
+  *value = container;
+}
+
+template <typename T>
+void SetMetric<T>::ToLegacyJson(rapidjson::Document* document) {
+  rapidjson::Value metric_list(rapidjson::kArrayType);
+  for (const T& s: value_) {
+    rapidjson::Value entry_value;
+    ToJsonValue(s, TUnit::NONE, document, &entry_value);
+    metric_list.PushBack(entry_value, document->GetAllocator());
+  }
+  rapidjson::Value key(key_.c_str(), document->GetAllocator());
+  document->AddMember(key, metric_list, document->GetAllocator());
+}
+
+template <typename T>
+string SetMetric<T>::ToHumanReadable() {
+  stringstream out;
+  PrettyPrinter::PrintStringList<set<T>>(value_, TUnit::NONE, &out);
+  return out.str();
+}
+
+template <typename T, int StatsSelection>
+TMetricKind::type StatsMetric<T, StatsSelection>::ToPrometheus(
+    string name, stringstream* val, stringstream* metric_kind) {
+  lock_guard<mutex> l(lock_);
+
+  *val << name << "_total " << boost::accumulators::count(acc_) << "\n";
+
+  if (boost::accumulators::count(acc_) > 0) {
+    if (IsUnitTimeBased(unit_)) {
+      *val << name << "_last " << ConvertToPrometheusSecs(value_, unit_) << "\n";
+    } else {
+      *val << name << "_last " << value_ << "\n";
+    }
+
+    if (StatsSelection & StatsType::MIN) {
+      if (IsUnitTimeBased(unit_)) {
+        *val << name << "_min "
+             << ConvertToPrometheusSecs(boost::accumulators::min(acc_), unit_) << "\n";
+      } else {
+        *val << name << "_min " << boost::accumulators::min(acc_) << "\n";
+      }
+    }
+
+    if (StatsSelection & StatsType::MAX) {
+      if (IsUnitTimeBased(unit_)) {
+        *val << name << "_max "
+             << ConvertToPrometheusSecs(boost::accumulators::max(acc_), unit_) << "\n";
+      } else {
+        *val << name << "_max " << boost::accumulators::max(acc_) << "\n";
+      }
+    }
+
+    if (StatsSelection & StatsType::MEAN) {
+      if (IsUnitTimeBased(unit_)) {
+        *val << name << "_mean "
+             << ConvertToPrometheusSecs(boost::accumulators::mean(acc_), unit_) << "\n";
+      } else {
+        *val << name << "_mean " << boost::accumulators::mean(acc_) << "\n";
+      }
+    }
+
+    if (StatsSelection & StatsType::STDDEV) {
+      if (IsUnitTimeBased(unit_)) {
+        *val << name << "_stddev "
+             << ConvertToPrometheusSecs(
+                    std::sqrt(boost::accumulators::variance(acc_)), unit_)
+             << "\n";
+      } else {
+        *val << name << "_stddev " << std::sqrt(boost::accumulators::variance(acc_))
+             << "\n";
+      }
+    }
+  }
+  *metric_kind << "# TYPE " << name << " counter";
+  return TMetricKind::STATS;
+}
+
+template <typename T, int StatsSelection>
+void StatsMetric<T, StatsSelection>::ToJson(
+    rapidjson::Document* document, rapidjson::Value* val) {
+  lock_guard<mutex> l(lock_);
+  rapidjson::Value container(rapidjson::kObjectType);
+  AddStandardFields(document, &container);
+  rapidjson::Value units(PrintThriftEnum(unit_).c_str(), document->GetAllocator());
+  container.AddMember("units", units, document->GetAllocator());
+
+  if (StatsSelection & StatsType::COUNT) {
+    container.AddMember("count",
+        static_cast<uint64_t>(boost::accumulators::count(acc_)),
+        document->GetAllocator());
+  }
+
+  if (boost::accumulators::count(acc_) > 0) {
+    container.AddMember("last", value_, document->GetAllocator());
+
+    if (StatsSelection & StatsType::MIN) {
+      container.AddMember("min",
+          static_cast<uint64_t>(boost::accumulators::min(acc_)),
+          document->GetAllocator());
+    }
+
+    if (StatsSelection & StatsType::MAX) {
+      container.AddMember("max", boost::accumulators::max(acc_),
+          document->GetAllocator());
+    }
+
+    if (StatsSelection & StatsType::MEAN) {
+      container.AddMember("mean", boost::accumulators::mean(acc_),
+        document->GetAllocator());
+    }
+
+    if (StatsSelection & StatsType::STDDEV) {
+      container.AddMember("stddev", sqrt(boost::accumulators::variance(acc_)),
+          document->GetAllocator());
+    }
+  }
+  *val = container;
+}
+
+template <typename T, int StatsSelection>
+void StatsMetric<T, StatsSelection>::ToLegacyJson(rapidjson::Document* document) {
+  stringstream ss;
+  lock_guard<mutex> l(lock_);
+  rapidjson::Value container(rapidjson::kObjectType);
+
+  if (StatsSelection & StatsType::COUNT) {
+    container.AddMember("count", boost::accumulators::count(acc_),
+        document->GetAllocator());
+  }
+
+  if (boost::accumulators::count(acc_) > 0) {
+    container.AddMember("last", value_, document->GetAllocator());
+    if (StatsSelection & StatsType::MIN) {
+      container.AddMember("min", boost::accumulators::min(acc_),
+          document->GetAllocator());
+    }
+
+    if (StatsSelection & StatsType::MAX) {
+      container.AddMember("max", boost::accumulators::max(acc_),
+          document->GetAllocator());
+    }
+
+    if (StatsSelection & StatsType::MEAN) {
+      container.AddMember("mean", boost::accumulators::mean(acc_),
+        document->GetAllocator());
+    }
+
+    if (StatsSelection & StatsType::STDDEV) {
+      container.AddMember("stddev", sqrt(boost::accumulators::variance(acc_)),
+          document->GetAllocator());
+    }
+  }
+  rapidjson::Value key(key_.c_str(), document->GetAllocator());
+  document->AddMember(key, container, document->GetAllocator());
+}
+
+template <typename T, int StatsSelection>
+string StatsMetric<T, StatsSelection>::ToHumanReadable() {
+  stringstream out;
+  if (StatsSelection & StatsType::COUNT) {
+    out << "count: " << boost::accumulators::count(acc_);
+    if (boost::accumulators::count(acc_) > 0) out << ", ";
+  }
+  if (boost::accumulators::count(acc_) > 0) {
+
+    out << "last: " << PrettyPrinter::Print(value_, unit_);
+    if (StatsSelection & StatsType::MIN) {
+      out << ", min: " << PrettyPrinter::Print(boost::accumulators::min(acc_), unit_);
+    }
+
+    if (StatsSelection & StatsType::MAX) {
+      out << ", max: " << PrettyPrinter::Print(boost::accumulators::max(acc_), unit_);
+    }
+
+    if (StatsSelection & StatsType::MEAN) {
+      out << ", mean: " << PrettyPrinter::Print(boost::accumulators::mean(acc_), unit_);
+    }
+
+    if (StatsSelection & StatsType::STDDEV) {
+      out << ", stddev: " << PrettyPrinter::Print(
+          sqrt(boost::accumulators::variance(acc_)), unit_);
+    }
+  }
+  return out.str();
+}
+
+// Explicitly instantiate template classes with parameter combinations that will be used.
+// If these classes are instantiated with new parameters, the instantiation must be
+// added to this list. This is required because some methods of these classes are
+// defined in .cc files and are used from other translation units.
+template class SetMetric<int32_t>;
+template class SetMetric<string>;
+template class StatsMetric<uint64_t, StatsType::ALL>;
+template class StatsMetric<double, StatsType::ALL>;
+template class StatsMetric<uint64_t, StatsType::MEAN>;
+template class StatsMetric<uint64_t, StatsType::MAX | StatsType::MEAN>;
+}
diff --git a/be/src/util/collection-metrics.h b/be/src/util/collection-metrics.h
index a299554..d4e883a 100644
--- a/be/src/util/collection-metrics.h
+++ b/be/src/util/collection-metrics.h
@@ -15,9 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_UTIL_COLLECTION_METRICS_H
-#define IMPALA_UTIL_COLLECTION_METRICS_H
+#pragma once
 
 #include "util/metrics.h"
 
@@ -32,8 +30,6 @@
 #include <boost/accumulators/statistics/max.hpp>
 #include <boost/accumulators/statistics/variance.hpp>
 
-#include "util/pretty-printer.h"
-
 namespace impala {
 
 /// Collection metrics are those whose values have more structure than simple
@@ -75,41 +71,16 @@ class SetMetric : public Metric {
   void Reset() { value_.clear(); }
 
   virtual TMetricKind::type ToPrometheus(
-      std::string name, std::stringstream* val, std::stringstream* metric_kind) {
+      std::string name, std::stringstream* val, std::stringstream* metric_kind) override {
     // this is not supported type in prometheus, so ignore
     return TMetricKind::SET;
   }
 
-  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* value) {
-    rapidjson::Value container(rapidjson::kObjectType);
-    AddStandardFields(document, &container);
-    rapidjson::Value metric_list(rapidjson::kArrayType);
-    for (const T& s: value_) {
-      rapidjson::Value entry_value;
-      ToJsonValue(s, TUnit::NONE, document, &entry_value);
-      metric_list.PushBack(entry_value, document->GetAllocator());
-    }
-    container.AddMember("items", metric_list, document->GetAllocator());
-    *value = container;
-  }
+  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* value) override;
 
-  virtual void ToLegacyJson(rapidjson::Document* document) {
-    rapidjson::Value metric_list(rapidjson::kArrayType);
-    for (const T& s: value_) {
-      rapidjson::Value entry_value;
-      ToJsonValue(s, TUnit::NONE, document, &entry_value);
-      metric_list.PushBack(entry_value, document->GetAllocator());
-    }
-    rapidjson::Value key(key_.c_str(), document->GetAllocator());
-    document->AddMember(key, metric_list, document->GetAllocator());
-  }
+  virtual void ToLegacyJson(rapidjson::Document* document) override;
 
-  virtual std::string ToHumanReadable() {
-    std::stringstream out;
-    PrettyPrinter::PrintStringList<std::set<T>>(
-        value_, TUnit::NONE, &out);
-    return out.str();
-  }
+  virtual std::string ToHumanReadable() override;
 
  private:
   /// Lock protecting the set
@@ -152,165 +123,13 @@ class StatsMetric : public Metric {
   }
 
   virtual TMetricKind::type ToPrometheus(
-      std::string name, std::stringstream* val, std::stringstream* metric_kind) {
-    boost::lock_guard<boost::mutex> l(lock_);
-
-    *val << name << "_total " << boost::accumulators::count(acc_) << "\n";
-
-    if (boost::accumulators::count(acc_) > 0) {
-      if (IsUnitTimeBased(unit_)) {
-        *val << name << "_last " << ConvertToPrometheusSecs(value_, unit_) << "\n";
-      } else {
-        *val << name << "_last " << value_ << "\n";
-      }
-
-      if (StatsSelection & StatsType::MIN) {
-        if (IsUnitTimeBased(unit_)) {
-          *val << name << "_min "
-               << ConvertToPrometheusSecs(boost::accumulators::min(acc_), unit_) << "\n";
-        } else {
-          *val << name << "_min " << boost::accumulators::min(acc_) << "\n";
-        }
-      }
+      std::string name, std::stringstream* val, std::stringstream* metric_kind) override;
 
-      if (StatsSelection & StatsType::MAX) {
-        if (IsUnitTimeBased(unit_)) {
-          *val << name << "_max "
-               << ConvertToPrometheusSecs(boost::accumulators::max(acc_), unit_) << "\n";
-        } else {
-          *val << name << "_max " << boost::accumulators::max(acc_) << "\n";
-        }
-      }
+  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* val) override;
 
-      if (StatsSelection & StatsType::MEAN) {
-        if (IsUnitTimeBased(unit_)) {
-          *val << name << "_mean "
-               << ConvertToPrometheusSecs(boost::accumulators::mean(acc_), unit_) << "\n";
-        } else {
-          *val << name << "_mean " << boost::accumulators::mean(acc_) << "\n";
-        }
-      }
+  virtual void ToLegacyJson(rapidjson::Document* document) override;
 
-      if (StatsSelection & StatsType::STDDEV) {
-        if (IsUnitTimeBased(unit_)) {
-          *val << name << "_stddev "
-               << ConvertToPrometheusSecs(
-                      std::sqrt(boost::accumulators::variance(acc_)), unit_)
-               << "\n";
-        } else {
-          *val << name << "_stddev " << std::sqrt(boost::accumulators::variance(acc_))
-               << "\n";
-        }
-      }
-    }
-    *metric_kind << "# TYPE " << name << " counter";
-    return TMetricKind::STATS;
-  }
-
-  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* val) {
-    boost::lock_guard<boost::mutex> l(lock_);
-    rapidjson::Value container(rapidjson::kObjectType);
-    AddStandardFields(document, &container);
-    rapidjson::Value units(PrintThriftEnum(unit_).c_str(), document->GetAllocator());
-    container.AddMember("units", units, document->GetAllocator());
-
-    if (StatsSelection & StatsType::COUNT) {
-      container.AddMember("count",
-          static_cast<uint64_t>(boost::accumulators::count(acc_)),
-          document->GetAllocator());
-    }
-
-    if (boost::accumulators::count(acc_) > 0) {
-      container.AddMember("last", value_, document->GetAllocator());
-
-      if (StatsSelection & StatsType::MIN) {
-        container.AddMember("min",
-            static_cast<uint64_t>(boost::accumulators::min(acc_)),
-            document->GetAllocator());
-      }
-
-      if (StatsSelection & StatsType::MAX) {
-        container.AddMember("max", boost::accumulators::max(acc_),
-            document->GetAllocator());
-      }
-
-      if (StatsSelection & StatsType::MEAN) {
-        container.AddMember("mean", boost::accumulators::mean(acc_),
-          document->GetAllocator());
-      }
-
-      if (StatsSelection & StatsType::STDDEV) {
-        container.AddMember("stddev", sqrt(boost::accumulators::variance(acc_)),
-            document->GetAllocator());
-      }
-    }
-    *val = container;
-  }
-
-  virtual void ToLegacyJson(rapidjson::Document* document) {
-    std::stringstream ss;
-    boost::lock_guard<boost::mutex> l(lock_);
-    rapidjson::Value container(rapidjson::kObjectType);
-
-    if (StatsSelection & StatsType::COUNT) {
-      container.AddMember("count", boost::accumulators::count(acc_),
-          document->GetAllocator());
-    }
-
-    if (boost::accumulators::count(acc_) > 0) {
-      container.AddMember("last", value_, document->GetAllocator());
-      if (StatsSelection & StatsType::MIN) {
-        container.AddMember("min", boost::accumulators::min(acc_),
-            document->GetAllocator());
-      }
-
-      if (StatsSelection & StatsType::MAX) {
-        container.AddMember("max", boost::accumulators::max(acc_),
-            document->GetAllocator());
-      }
-
-      if (StatsSelection & StatsType::MEAN) {
-        container.AddMember("mean", boost::accumulators::mean(acc_),
-          document->GetAllocator());
-      }
-
-      if (StatsSelection & StatsType::STDDEV) {
-        container.AddMember("stddev", sqrt(boost::accumulators::variance(acc_)),
-            document->GetAllocator());
-      }
-    }
-    rapidjson::Value key(key_.c_str(), document->GetAllocator());
-    document->AddMember(key, container, document->GetAllocator());
-  }
-
-  virtual std::string ToHumanReadable() {
-    std::stringstream out;
-    if (StatsSelection & StatsType::COUNT) {
-      out << "count: " << boost::accumulators::count(acc_);
-      if (boost::accumulators::count(acc_) > 0) out << ", ";
-    }
-    if (boost::accumulators::count(acc_) > 0) {
-
-      out << "last: " << PrettyPrinter::Print(value_, unit_);
-      if (StatsSelection & StatsType::MIN) {
-        out << ", min: " << PrettyPrinter::Print(boost::accumulators::min(acc_), unit_);
-      }
-
-      if (StatsSelection & StatsType::MAX) {
-        out << ", max: " << PrettyPrinter::Print(boost::accumulators::max(acc_), unit_);
-      }
-
-      if (StatsSelection & StatsType::MEAN) {
-        out << ", mean: " << PrettyPrinter::Print(boost::accumulators::mean(acc_), unit_);
-      }
-
-      if (StatsSelection & StatsType::STDDEV) {
-        out << ", stddev: " << PrettyPrinter::Print(
-            sqrt(boost::accumulators::variance(acc_)), unit_);
-      }
-    }
-    return out.str();
-  }
+  virtual std::string ToHumanReadable() override;
 
  private:
   /// The units of the values captured in this metric, used when pretty-printing.
@@ -333,6 +152,11 @@ class StatsMetric : public Metric {
 
 };
 
-};
-
-#endif
+// These template classes are instantiated in the .cc file.
+extern template class SetMetric<int32_t>;
+extern template class SetMetric<string>;
+extern template class StatsMetric<uint64_t, StatsType::ALL>;
+extern template class StatsMetric<double, StatsType::ALL>;
+extern template class StatsMetric<uint64_t, StatsType::MEAN>;
+extern template class StatsMetric<uint64_t, StatsType::MAX | StatsType::MEAN>;
+}
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index 010af7a..0d5f493 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -38,6 +38,7 @@
 #include "util/jni-util.h"
 #include "util/mem-info.h"
 #include "util/pprof-path-handlers.h"
+#include "util/pretty-printer.h"
 #include "util/process-state-info.h"
 
 #include "common/names.h"
diff --git a/be/src/util/histogram-metric.cc b/be/src/util/histogram-metric.cc
new file mode 100644
index 0000000..012711d
--- /dev/null
+++ b/be/src/util/histogram-metric.cc
@@ -0,0 +1,177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/histogram-metric.h"
+
+#include "kudu/util/hdr_histogram.h"
+#include "util/json-util.h"
+#include "util/pretty-printer.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+HistogramMetric::HistogramMetric(
+    const TMetricDef& def, uint64_t highest_trackable_value, int num_significant_digits)
+  : Metric(def),
+    histogram_(new HdrHistogram(highest_trackable_value, num_significant_digits)),
+    unit_(def.units) {
+  DCHECK_EQ(TMetricKind::HISTOGRAM, def.kind);
+}
+
+void HistogramMetric::Reset() {
+  lock_guard<SpinLock> l(lock_);
+  uint64_t highest = histogram_->highest_trackable_value();
+  int digits = histogram_->num_significant_digits();
+  histogram_.reset(new HdrHistogram(highest, digits));
+}
+
+void HistogramMetric::ToJson(rapidjson::Document* document, rapidjson::Value* value) {
+  rapidjson::Value container(rapidjson::kObjectType);
+  AddStandardFields(document, &container);
+
+  {
+    lock_guard<SpinLock> l(lock_);
+
+    container.AddMember(
+        "25th %-ile", histogram_->ValueAtPercentile(25), document->GetAllocator());
+    container.AddMember(
+        "50th %-ile", histogram_->ValueAtPercentile(50), document->GetAllocator());
+    container.AddMember(
+        "75th %-ile", histogram_->ValueAtPercentile(75), document->GetAllocator());
+    container.AddMember(
+        "90th %-ile", histogram_->ValueAtPercentile(90), document->GetAllocator());
+    container.AddMember(
+        "95th %-ile", histogram_->ValueAtPercentile(95), document->GetAllocator());
+    container.AddMember(
+        "99.9th %-ile", histogram_->ValueAtPercentile(99.9), document->GetAllocator());
+    container.AddMember("max", histogram_->MaxValue(), document->GetAllocator());
+    container.AddMember("min", histogram_->MinValue(), document->GetAllocator());
+    container.AddMember("count", histogram_->TotalCount(), document->GetAllocator());
+  }
+  rapidjson::Value type_value(
+      PrintThriftEnum(TMetricKind::HISTOGRAM).c_str(), document->GetAllocator());
+  container.AddMember("kind", type_value, document->GetAllocator());
+  rapidjson::Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
+  container.AddMember("units", units, document->GetAllocator());
+
+  *value = container;
+}
+
+TMetricKind::type HistogramMetric::ToPrometheus(
+    string name, stringstream* value, stringstream* metric_kind) {
+  {
+    lock_guard<SpinLock> l(lock_);
+
+    // check if unit its 'TIME_MS','TIME_US' or 'TIME_NS' and convert it to seconds,
+    // this is because prometheus only supports time format in seconds
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "{le=\"0.2\"} "
+             << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(25), unit_) << "\n";
+    } else {
+      *value << name << "{le=\"0.2\"} " << histogram_->ValueAtPercentile(25) << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "{le=\"0.5\"} "
+             << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(50), unit_) << "\n";
+    } else {
+      *value << name << "{le=\"0.5\"} " << histogram_->ValueAtPercentile(50) << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "{le=\"0.7\"} "
+             << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(75), unit_) << "\n";
+    } else {
+      *value << name << "{le=\"0.7\"} " << histogram_->ValueAtPercentile(75) << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "{le=\"0.9\"} "
+             << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(90), unit_) << "\n";
+    } else {
+      *value << name << "{le=\"0.9\"} " << histogram_->ValueAtPercentile(90) << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "{le=\"0.95\"} "
+             << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(95), unit_) << "\n";
+    } else {
+      *value << name << "{le=\"0.95\"} " << histogram_->ValueAtPercentile(95) << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "{le=\"0.999\"} "
+             << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(99.9), unit_)
+             << "\n";
+    } else {
+      *value << name << "{le=\"0.999\"} " << histogram_->ValueAtPercentile(99.9) << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "_max " << ConvertToPrometheusSecs(histogram_->MaxValue(), unit_)
+             << "\n";
+    } else {
+      *value << name << "_max " << histogram_->MaxValue() << "\n";
+    }
+
+    if (IsUnitTimeBased(unit_)) {
+      *value << name << "_min " << ConvertToPrometheusSecs(histogram_->MinValue(), unit_)
+             << "\n";
+    } else {
+      *value << name << "_min " << histogram_->MinValue() << "\n";
+    }
+
+    *value << name << "_count " << histogram_->TotalCount();
+  }
+
+  *metric_kind << "# TYPE " << name << " histogram";
+  return TMetricKind::HISTOGRAM;
+}
+
+string HistogramMetric::ToHumanReadable() {
+  lock_guard<SpinLock> l(lock_);
+  return HistogramToHumanReadable(histogram_.get(), unit_);
+}
+
+template <class T>
+string HistogramMetric::HistogramToHumanReadable(T* histogram, TUnit::type unit) {
+  DCHECK(histogram != nullptr);
+  stringstream out;
+  out << "Count: " << histogram->TotalCount() << ", "
+      << "min / max: " << PrettyPrinter::Print(histogram->MinValue(), unit) << " / "
+      << PrettyPrinter::Print(histogram->MaxValue(), unit) << ", "
+      << "25th %-ile: " << PrettyPrinter::Print(histogram->ValueAtPercentile(25), unit)
+      << ", "
+      << "50th %-ile: " << PrettyPrinter::Print(histogram->ValueAtPercentile(50), unit)
+      << ", "
+      << "75th %-ile: " << PrettyPrinter::Print(histogram->ValueAtPercentile(75), unit)
+      << ", "
+      << "90th %-ile: " << PrettyPrinter::Print(histogram->ValueAtPercentile(90), unit)
+      << ", "
+      << "95th %-ile: " << PrettyPrinter::Print(histogram->ValueAtPercentile(95), unit)
+      << ", "
+      << "99.9th %-ile: "
+      << PrettyPrinter::Print(histogram->ValueAtPercentile(99.9), unit);
+  return out.str();
+}
+
+// Instantiate the template for both Impala and Kudu histograms.
+template string HistogramMetric::HistogramToHumanReadable<>(HdrHistogram*, TUnit::type);
+template string HistogramMetric::HistogramToHumanReadable<>(
+    kudu::HdrHistogram*, TUnit::type);
+} // namespace impala
diff --git a/be/src/util/histogram-metric.h b/be/src/util/histogram-metric.h
index ca4499f..7aac0f1 100644
--- a/be/src/util/histogram-metric.h
+++ b/be/src/util/histogram-metric.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef IMPALA_UTIL_HISTOGRAM_METRIC
-#define IMPALA_UTIL_HISTOGRAM_METRIC
+#pragma once
 
 #include "util/hdr-histogram.h"
 #include "util/metrics.h"
@@ -24,6 +23,8 @@
 
 namespace impala {
 
+class HdrHistogram;
+
 /// Metric which constructs (using HdrHistogram) a histogram of a set of values.
 /// Thread-safe: histogram access protected by a spin lock.
 class HistogramMetric : public Metric {
@@ -31,122 +32,13 @@ class HistogramMetric : public Metric {
   /// Constructs a new histogram metric. `highest_trackable_value` is the maximum value
   /// that may be entered into the histogram. `num_significant_digits` is the precision
   /// that values must be stored with.
-  HistogramMetric(
-      const TMetricDef& def, uint64_t highest_trackable_value, int num_significant_digits)
-    : Metric(def),
-      histogram_(new HdrHistogram(highest_trackable_value, num_significant_digits)),
-      unit_(def.units) {
-    DCHECK_EQ(TMetricKind::HISTOGRAM, def.kind);
-  }
-
-  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* value) override {
-    rapidjson::Value container(rapidjson::kObjectType);
-    AddStandardFields(document, &container);
+  HistogramMetric(const TMetricDef& def, uint64_t highest_trackable_value,
+      int num_significant_digits);
 
-    {
-      boost::lock_guard<SpinLock> l(lock_);
-
-      container.AddMember(
-          "25th %-ile", histogram_->ValueAtPercentile(25), document->GetAllocator());
-      container.AddMember(
-          "50th %-ile", histogram_->ValueAtPercentile(50), document->GetAllocator());
-      container.AddMember(
-          "75th %-ile", histogram_->ValueAtPercentile(75), document->GetAllocator());
-      container.AddMember(
-          "90th %-ile", histogram_->ValueAtPercentile(90), document->GetAllocator());
-      container.AddMember(
-          "95th %-ile", histogram_->ValueAtPercentile(95), document->GetAllocator());
-      container.AddMember(
-          "99.9th %-ile", histogram_->ValueAtPercentile(99.9), document->GetAllocator());
-      container.AddMember("max", histogram_->MaxValue(), document->GetAllocator());
-      container.AddMember("min", histogram_->MinValue(), document->GetAllocator());
-      container.AddMember("count", histogram_->TotalCount(), document->GetAllocator());
-    }
-    rapidjson::Value type_value(PrintThriftEnum(TMetricKind::HISTOGRAM).c_str(),
-        document->GetAllocator());
-    container.AddMember("kind", type_value, document->GetAllocator());
-    rapidjson::Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
-    container.AddMember("units", units, document->GetAllocator());
-
-    *value = container;
-  }
+  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* value) override;
 
   virtual TMetricKind::type ToPrometheus(std::string name, std::stringstream* value,
-      std::stringstream* metric_kind) override {
-    {
-      boost::lock_guard<SpinLock> l(lock_);
-
-      // check if unit its 'TIME_MS','TIME_US' or 'TIME_NS' and convert it to seconds,
-      // this is because prometheus only supports time format in seconds
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "{le=\"0.2\"} "
-               << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(25), unit_)
-               << "\n";
-      } else {
-        *value << name << "{le=\"0.2\"} " << histogram_->ValueAtPercentile(25) << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "{le=\"0.5\"} "
-               << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(50), unit_)
-               << "\n";
-      } else {
-        *value << name << "{le=\"0.5\"} " << histogram_->ValueAtPercentile(50) << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "{le=\"0.7\"} "
-               << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(75), unit_)
-               << "\n";
-      } else {
-        *value << name << "{le=\"0.7\"} " << histogram_->ValueAtPercentile(75) << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "{le=\"0.9\"} "
-               << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(90), unit_)
-               << "\n";
-      } else {
-        *value << name << "{le=\"0.9\"} " << histogram_->ValueAtPercentile(90) << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "{le=\"0.95\"} "
-               << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(95), unit_)
-               << "\n";
-      } else {
-        *value << name << "{le=\"0.95\"} " << histogram_->ValueAtPercentile(95) << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "{le=\"0.999\"} "
-               << ConvertToPrometheusSecs(histogram_->ValueAtPercentile(99.9), unit_)
-               << "\n";
-      } else {
-        *value << name << "{le=\"0.999\"} " << histogram_->ValueAtPercentile(99.9)
-               << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "_max "
-               << ConvertToPrometheusSecs(histogram_->MaxValue(), unit_) << "\n";
-      } else {
-        *value << name << "_max " << histogram_->MaxValue() << "\n";
-      }
-
-      if (IsUnitTimeBased(unit_)) {
-        *value << name << "_min "
-               << ConvertToPrometheusSecs(histogram_->MinValue(), unit_) << "\n";
-      } else {
-        *value << name << "_min " << histogram_->MinValue() << "\n";
-      }
-
-      *value << name << "_count " << histogram_->TotalCount();
-    }
-
-    *metric_kind << "# TYPE " << name << " histogram";
-    return TMetricKind::HISTOGRAM;
-  }
+      std::stringstream* metric_kind) override;
 
   void Update(int64_t val) {
     boost::lock_guard<SpinLock> l(lock_);
@@ -154,46 +46,19 @@ class HistogramMetric : public Metric {
   }
 
   /// Reset the histogram by removing all previous entries.
-  void Reset() {
-    boost::lock_guard<SpinLock> l(lock_);
-    uint64_t highest = histogram_->highest_trackable_value();
-    int digits = histogram_->num_significant_digits();
-    histogram_.reset(new HdrHistogram(highest, digits));
-  }
+  void Reset();
 
   virtual void ToLegacyJson(rapidjson::Document*) override {}
 
   const TUnit::type& unit() const { return unit_; }
 
-  virtual std::string ToHumanReadable() override {
-    boost::lock_guard<SpinLock> l(lock_);
-    return HistogramToHumanReadable(histogram_.get(), unit_);
-  }
+  virtual std::string ToHumanReadable() override;
 
   /// Render a HdrHistogram into a human readable string representation. The histogram
   /// type is a template parameter so that it accepts both Impala's and Kudu's
   /// HdrHistogram classes.
   template <class T>
-  static std::string HistogramToHumanReadable(T* histogram, TUnit::type unit) {
-    DCHECK(histogram != nullptr);
-    std::stringstream out;
-    out << "Count: " << histogram->TotalCount() << ", "
-        << "min / max: " << PrettyPrinter::Print(histogram->MinValue(), unit)
-        << " / " << PrettyPrinter::Print(histogram->MaxValue(), unit) << ", "
-        << "25th %-ile: "
-        << PrettyPrinter::Print(histogram->ValueAtPercentile(25), unit) << ", "
-        << "50th %-ile: "
-        << PrettyPrinter::Print(histogram->ValueAtPercentile(50), unit) << ", "
-        << "75th %-ile: "
-        << PrettyPrinter::Print(histogram->ValueAtPercentile(75), unit) << ", "
-        << "90th %-ile: "
-        << PrettyPrinter::Print(histogram->ValueAtPercentile(90), unit) << ", "
-        << "95th %-ile: "
-        << PrettyPrinter::Print(histogram->ValueAtPercentile(95), unit) << ", "
-        << "99.9th %-ile: "
-        << PrettyPrinter::Print(histogram->ValueAtPercentile(99.9), unit);
-    return out.str();
-  }
+  static std::string HistogramToHumanReadable(T* histogram, TUnit::type unit);
 
  private:
   /// Protects histogram_ pointer itself.
@@ -203,7 +68,4 @@ class HistogramMetric : public Metric {
 
   DISALLOW_COPY_AND_ASSIGN(HistogramMetric);
 };
-
 }
-
-#endif
diff --git a/be/src/util/metrics-test.cc b/be/src/util/metrics-test.cc
index 3302fb3..b97d0e8 100644
--- a/be/src/util/metrics-test.cc
+++ b/be/src/util/metrics-test.cc
@@ -232,9 +232,9 @@ TEST_F(MetricsTest, StatsMetrics) {
 
 TEST_F(MetricsTest, StatsMetricsSingle) {
   MetricGroup metrics("StatsMetrics");
-  StatsMetric<int, StatsType::MAX | StatsType::MEAN>*
+  StatsMetric<uint64_t, StatsType::MAX | StatsType::MEAN>*
       stats_metric =
-      StatsMetric<int, StatsType::MAX | StatsType::MEAN>::CreateAndRegister(&metrics,
+      StatsMetric<uint64_t, StatsType::MAX | StatsType::MEAN>::CreateAndRegister(&metrics,
           "impala-server.io.mgr.cached-file-handles-hit-ratio");
   EXPECT_EQ(stats_metric->ToHumanReadable(), "");
   stats_metric->Update(3);
diff --git a/be/src/util/metrics.cc b/be/src/util/metrics.cc
index 2a9cbf6..e9ec5b8 100644
--- a/be/src/util/metrics.cc
+++ b/be/src/util/metrics.cc
@@ -18,6 +18,8 @@
 #include "util/metrics.h"
 
 #include <sstream>
+#include <stack>
+
 #include <boost/algorithm/string/join.hpp>
 #include <boost/bind.hpp>
 #include <boost/mem_fn.hpp>
@@ -28,6 +30,9 @@
 
 #include "common/logging.h"
 #include "util/impalad-metrics.h"
+#include "util/json-util.h"
+#include "util/pretty-printer.h"
+#include "util/webserver.h"
 
 #include "common/names.h"
 
@@ -45,8 +50,6 @@ void ToJsonValue<string>(const string& value, const TUnit::type unit,
   *out_val = val;
 }
 
-}
-
 void Metric::AddStandardFields(Document* document, Value* val) {
   Value name(key_.c_str(), document->GetAllocator());
   val->AddMember("name", name, document->GetAllocator());
@@ -56,6 +59,59 @@ void Metric::AddStandardFields(Document* document, Value* val) {
   val->AddMember("human_readable", metric_value, document->GetAllocator());
 }
 
+template <typename T, TMetricKind::type metric_kind_t>
+void ScalarMetric<T, metric_kind_t>::ToJson(Document* document, Value* val) {
+  Value container(kObjectType);
+  AddStandardFields(document, &container);
+
+  Value metric_value;
+  ToJsonValue(GetValue(), TUnit::NONE, document, &metric_value);
+  container.AddMember("value", metric_value, document->GetAllocator());
+
+  Value type_value(PrintThriftEnum(kind()).c_str(), document->GetAllocator());
+  container.AddMember("kind", type_value, document->GetAllocator());
+  Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
+  container.AddMember("units", units, document->GetAllocator());
+  *val = container;
+}
+
+template <typename T, TMetricKind::type metric_kind_t>
+void ScalarMetric<T, metric_kind_t>::ToLegacyJson(Document* document) {
+  Value val;
+  ToJsonValue(GetValue(), TUnit::NONE, document, &val);
+  Value key(key_.c_str(), document->GetAllocator());
+  document->AddMember(key, val, document->GetAllocator());
+}
+
+template <typename T, TMetricKind::type metric_kind_t>
+TMetricKind::type ScalarMetric<T, metric_kind_t>::ToPrometheus(
+    string name, stringstream* val, stringstream* metric_kind) {
+  string metric_type = PrintThriftEnum(kind()).c_str();
+  // prometheus doesn't support 'property', so ignore it
+  if (!metric_type.compare("property")) {
+    return TMetricKind::PROPERTY;
+  }
+
+  if (IsUnitTimeBased(unit())) {
+    // check if unit its 'TIME_MS','TIME_US' or 'TIME_NS' and convert it to seconds,
+    // this is because prometheus only supports time format in seconds
+    *val << ConvertToPrometheusSecs(GetValue(), unit());
+  } else {
+    *val << GetValue();
+  }
+
+  // convert metric type to lower case, that's what prometheus expects
+  std::transform(metric_type.begin(), metric_type.end(), metric_type.begin(), ::tolower);
+
+  *metric_kind << "# TYPE " << name << " " << metric_type;
+  return kind();
+}
+
+template <typename T, TMetricKind::type metric_kind_t>
+string ScalarMetric<T, metric_kind_t>::ToHumanReadable() {
+  return PrettyPrinter::Print(GetValue(), unit());
+}
+
 MetricDefs* MetricDefs::GetInstance() {
   // Note that this is not thread-safe in C++03 (but will be in C++11 see
   // http://stackoverflow.com/a/19907903/132034). We don't bother with the double-check
@@ -268,6 +324,22 @@ MetricGroup* MetricGroup::FindChildGroup(const string& name) {
   return NULL;
 }
 
+Metric* MetricGroup::FindMetricForTestingInternal(const string& key) {
+  stack<MetricGroup*> groups;
+  groups.push(this);
+  lock_guard<SpinLock> l(lock_);
+  do {
+    MetricGroup* group = groups.top();
+    groups.pop();
+    auto it = group->metric_map_.find(key);
+    if (it != group->metric_map_.end()) return it->second;
+    for (const auto& child : group->children_) {
+      groups.push(child.second);
+    }
+  } while (!groups.empty());
+  return nullptr;
+}
+
 string MetricGroup::DebugString() {
   Webserver::WebRequest empty_req;
   Document document;
@@ -279,11 +351,45 @@ string MetricGroup::DebugString() {
   return strbuf.GetString();
 }
 
-TMetricDef impala::MakeTMetricDef(const string& key, TMetricKind::type kind,
-    TUnit::type unit) {
+TMetricDef MakeTMetricDef(const string& key, TMetricKind::type kind, TUnit::type unit) {
   TMetricDef ret;
   ret.__set_key(key);
   ret.__set_kind(kind);
   ret.__set_units(unit);
   return ret;
 }
+
+template <typename T>
+double ConvertToPrometheusSecs(const T& val, TUnit::type unit) {
+  double value = val;
+  if (unit == TUnit::type::TIME_MS) {
+    value /= 1000;
+  } else if (unit == TUnit::type::TIME_US) {
+    value /= 1000000;
+  } else if (unit == TUnit::type::TIME_NS) {
+    value /= 1000000000;
+  }
+  return value;
+}
+
+// Explicitly instantiate the variants that will be used.
+template double ConvertToPrometheusSecs<>(const double&, TUnit::type);
+template double ConvertToPrometheusSecs<>(const int64_t&, TUnit::type);
+template double ConvertToPrometheusSecs<>(const uint64_t&, TUnit::type);
+
+template <>
+double ConvertToPrometheusSecs<string>(const string& val, TUnit::type unit) {
+  DCHECK(false) << "Should not be called for string metrics";
+  return 0.0;
+}
+
+// Explicitly instantiate template classes with parameter combinations that will be used.
+// If these classes are instantiated with new parameters, the instantiation must be
+// added to this list. This is required because some methods of these classes are
+// defined in .cc files and are used from other translation units.
+template class LockedMetric<bool, TMetricKind::PROPERTY>;
+template class LockedMetric<std::string, TMetricKind::PROPERTY>;
+template class LockedMetric<double, TMetricKind::GAUGE>;
+template class AtomicMetric<TMetricKind::GAUGE>;
+template class AtomicMetric<TMetricKind::COUNTER>;
+} // namespace impala
diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h
index 7c7831d..69c2d3c 100644
--- a/be/src/util/metrics.h
+++ b/be/src/util/metrics.h
@@ -19,22 +19,20 @@
 
 #include <map>
 #include <sstream>
-#include <stack>
 #include <string>
 #include <vector>
 #include <boost/function.hpp>
 #include <boost/scoped_ptr.hpp>
 #include <boost/thread/locks.hpp>
 #include <gtest/gtest_prod.h> // for FRIEND_TEST
+#include <rapidjson/fwd.h>
 
 #include "common/atomic.h"
 #include "common/logging.h"
 #include "common/object-pool.h"
 #include "common/status.h"
 #include "util/debug-util.h"
-#include "util/json-util.h"
 #include "util/metrics-fwd.h"
-#include "util/pretty-printer.h"
 #include "util/spinlock.h"
 #include "util/webserver.h"
 
@@ -132,26 +130,6 @@ class Metric {
   void AddStandardFields(rapidjson::Document* document, rapidjson::Value* val);
 };
 
-template <typename T>
-inline double ConvertToPrometheusSecs(const T& val, TUnit::type unit) {
-  double value = val;
-  if (unit == TUnit::type::TIME_MS) {
-    value /= 1000;
-  } else if (unit == TUnit::type::TIME_US) {
-    value /= 1000000;
-  } else if (unit == TUnit::type::TIME_NS) {
-    value /= 1000000000;
-  }
-  return value;
-}
-
-template <>
-inline double ConvertToPrometheusSecs<std::string>(
-    const std::string& val, TUnit::type unit) {
-  DCHECK(false) << "Should not be called for string metrics";
-  return 0.0;
-}
-
 /// A ScalarMetric has a value which is a simple primitive type: e.g. integers, strings
 /// and floats. It is parameterised not only by the type of its value, but by both the
 /// unit (e.g. bytes/s), drawn from TUnit and the 'kind' of the metric itself.
@@ -177,55 +155,14 @@ class ScalarMetric: public Metric {
   /// Returns the current value. Thread-safe.
   virtual T GetValue() = 0;
 
-  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* val) override {
-    rapidjson::Value container(rapidjson::kObjectType);
-    AddStandardFields(document, &container);
-
-    rapidjson::Value metric_value;
-    ToJsonValue(GetValue(), TUnit::NONE, document, &metric_value);
-    container.AddMember("value", metric_value, document->GetAllocator());
-
-    rapidjson::Value type_value(PrintThriftEnum(kind()).c_str(), document->GetAllocator());
-    container.AddMember("kind", type_value, document->GetAllocator());
-    rapidjson::Value units(PrintThriftEnum(unit()).c_str(), document->GetAllocator());
-    container.AddMember("units", units, document->GetAllocator());
-    *val = container;
-  }
+  virtual void ToJson(rapidjson::Document* document, rapidjson::Value* val) override;
 
   virtual TMetricKind::type ToPrometheus(
-      std::string name, std::stringstream* val, std::stringstream* metric_kind) override {
-    std::string metric_type = PrintThriftEnum(kind()).c_str();
-    // prometheus doesn't support 'property', so ignore it
-    if (!metric_type.compare("property")) {
-      return TMetricKind::PROPERTY;
-    }
+      std::string name, std::stringstream* val, std::stringstream* metric_kind) override;
 
-    if (IsUnitTimeBased(unit())) {
-      // check if unit its 'TIME_MS','TIME_US' or 'TIME_NS' and convert it to seconds,
-      // this is because prometheus only supports time format in seconds
-      *val << ConvertToPrometheusSecs(GetValue(), unit());
-    } else {
-      *val << GetValue();
-    }
+  virtual std::string ToHumanReadable() override;
 
-    // convert metric type to lower case, that's what prometheus expects
-    std::transform(
-        metric_type.begin(), metric_type.end(), metric_type.begin(), ::tolower);
-
-    *metric_kind << "# TYPE " << name << " " << metric_type;
-    return kind();
-  }
-
-  virtual std::string ToHumanReadable() override {
-    return PrettyPrinter::Print(GetValue(), unit());
-  }
-
-  virtual void ToLegacyJson(rapidjson::Document* document) override {
-    rapidjson::Value val;
-    ToJsonValue(GetValue(), TUnit::NONE, document, &val);
-    rapidjson::Value key(key_.c_str(), document->GetAllocator());
-    document->AddMember(key, val, document->GetAllocator());
-  }
+  virtual void ToLegacyJson(rapidjson::Document* document) override;
 
   TUnit::type unit() const { return unit_; }
   TMetricKind::type kind() const { return metric_kind_t; }
@@ -465,19 +402,7 @@ class MetricGroup {
   /// Used for testing only.
   template <typename M>
   M* FindMetricForTesting(const std::string& key) {
-    std::stack<MetricGroup*> groups;
-    groups.push(this);
-    boost::lock_guard<SpinLock> l(lock_);
-    do {
-      MetricGroup* group = groups.top();
-      groups.pop();
-      MetricMap::const_iterator it = group->metric_map_.find(key);
-      if (it != group->metric_map_.end()) return reinterpret_cast<M*>(it->second);
-      for (const ChildGroupMap::value_type& child: group->children_) {
-        groups.push(child.second);
-      }
-    } while (!groups.empty());
-    return NULL;
+    return reinterpret_cast<M*>(FindMetricForTestingInternal(key));
   }
 
   /// Register page callbacks with the webserver. Only the root of any metric group
@@ -538,6 +463,9 @@ class MetricGroup {
   /// If args contains a paramater 'metric', only the json for that metric is returned.
   void CMCompatibleCallback(const Webserver::WebRequest& req,
       rapidjson::Document* document);
+
+  /// Non-templated implementation for FindMetricForTesting() that does not cast.
+  Metric* FindMetricForTestingInternal(const std::string& key);
 };
 
 
@@ -547,4 +475,16 @@ class MetricGroup {
 /// in special cases where the regular approach is unsuitable.
 TMetricDef MakeTMetricDef(const std::string& key, TMetricKind::type kind,
     TUnit::type unit);
+
+/// Helper to convert a value 'val' that is a time metric into fractional seconds
+/// (the only unit of time supported by Prometheus).
+template <typename T>
+double ConvertToPrometheusSecs(const T& val, TUnit::type unit);
+
+// These template classes are instantiated in the .cc file.
+extern template class LockedMetric<bool, TMetricKind::PROPERTY>;
+extern template class LockedMetric<std::string, TMetricKind::PROPERTY>;
+extern template class LockedMetric<double, TMetricKind::GAUGE>;
+extern template class AtomicMetric<TMetricKind::GAUGE>;
+extern template class AtomicMetric<TMetricKind::COUNTER>;
 }
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 4de033b..331c6d9 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -17,20 +17,21 @@
 
 #include "util/webserver.h"
 
+#include <signal.h>
+#include <stdio.h>
+#include <fstream>
+#include <map>
+#include <string>
 #include <boost/algorithm/string.hpp>
 #include <boost/bind.hpp>
 #include <boost/filesystem.hpp>
 #include <boost/lexical_cast.hpp>
 #include <boost/mem_fn.hpp>
 #include <boost/thread/locks.hpp>
-#include <fstream>
 #include <gutil/strings/substitute.h>
-#include <map>
-#include <stdio.h>
-#include <signal.h>
-#include <string>
-#include <rapidjson/stringbuffer.h>
+#include <rapidjson/document.h>
 #include <rapidjson/prettywriter.h>
+#include <rapidjson/stringbuffer.h>
 
 #include "common/logging.h"
 #include "rpc/thrift-util.h"
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index bf8f1a0..2fbc0cd 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -15,15 +15,13 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#pragma once
 
-#ifndef IMPALA_UTIL_WEBSERVER_H
-#define IMPALA_UTIL_WEBSERVER_H
-
-#include <boost/function.hpp>
-#include <boost/thread/shared_mutex.hpp>
 #include <map>
-#include <rapidjson/document.h>
 #include <string>
+#include <boost/function.hpp>
+#include <boost/thread/shared_mutex.hpp>
+#include <rapidjson/fwd.h>
 
 #include "common/status.h"
 #include "kudu/util/web_callback_registry.h"
@@ -204,5 +202,3 @@ class Webserver {
 };
 
 }
-
-#endif // IMPALA_UTIL_WEBSERVER_H