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 2017/08/02 06:28:29 UTC

[6/6] incubator-impala git commit: IMPALA-5658: addtl. process/system-wide memory metrics

IMPALA-5658: addtl. process/system-wide memory metrics

This is meant to help understand how the process is using memory, to
answer questions like:
* Is the Impala daemon using transparent huge pages?
* What is the system THP configuration?
* What is the RSS of the process?
* What is the virtual memory size of the process?

Most of these questions can be answered via other command line or
monitoring, except for the THP usage by the process, but adding
metrics helps consolidate the information in one place.

This commit adds a memory maintenance thread that periodically scrapes
information from the /proc and /sys filesystems and updates the
metrics.

The interfaces used are:
* /proc/<pid>/smaps, which is a documented interface to get detailed
  information about mapped memory:
  https://www.kernel.org/doc/Documentation/filesystems/proc.txt
* /sys/kernel/mm/ config values for transparent huge pages

Change-Id: I13873e305ba464d11dea0d7244a29ff4f332f1a9
Reviewed-on: http://gerrit.cloudera.org:8080/7472
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/3f82d157
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/3f82d157
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/3f82d157

Branch: refs/heads/master
Commit: 3f82d15704160f31d4060123520c3fb3493a3533
Parents: 507bd8b
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Jul 19 14:44:00 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Aug 2 04:06:57 2017 +0000

----------------------------------------------------------------------
 be/src/common/init.cc                |  30 +++++----
 be/src/runtime/exec-env.cc           |   2 +-
 be/src/util/default-path-handlers.cc |  11 ++++
 be/src/util/mem-info.cc              | 102 ++++++++++++++++++++++++++++--
 be/src/util/mem-info.h               |  48 +++++++++++++-
 be/src/util/memory-metrics.cc        |  75 +++++++++++++++++-----
 be/src/util/memory-metrics.h         |  35 +++++++++-
 common/thrift/metrics.json           |  84 ++++++++++++++++++++++++
 www/memz.tmpl                        |  35 ++++++++++
 9 files changed, 384 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/common/init.cc
----------------------------------------------------------------------
diff --git a/be/src/common/init.cc b/be/src/common/init.cc
index f05a995..797df39 100644
--- a/be/src/common/init.cc
+++ b/be/src/common/init.cc
@@ -128,21 +128,27 @@ static scoped_ptr<impala::Thread> pause_monitor;
   while (true) {
     SleepForMs(FLAGS_memory_maintenance_sleep_time_ms);
     impala::ExecEnv* env = impala::ExecEnv::GetInstance();
-    if (env == nullptr) continue; // ExecEnv may not have been created yet.
-    BufferPool* buffer_pool = env->buffer_pool();
-    if (buffer_pool != nullptr) buffer_pool->Maintenance();
+    // ExecEnv may not have been created yet or this may be the catalogd or statestored,
+    // which don't have ExecEnvs.
+    if (env != nullptr) {
+      BufferPool* buffer_pool = env->buffer_pool();
+      if (buffer_pool != nullptr) buffer_pool->Maintenance();
 
 #ifndef ADDRESS_SANITIZER
-    // When using tcmalloc, the process limit as measured by our trackers will
-    // be out of sync with the process usage. The metric is refreshed whenever
-    // memory is consumed or released via a MemTracker, so on a system with
-    // queries executing it will be refreshed frequently. However if the system
-    // is idle, we need to refresh the tracker occasionally since untracked
-    // memory may be allocated or freed, e.g. by background threads.
-    if (env != NULL && env->process_mem_tracker() != NULL) {
-      env->process_mem_tracker()->RefreshConsumptionFromMetric();
-    }
+      // When using tcmalloc, the process limit as measured by our trackers will
+      // be out of sync with the process usage. The metric is refreshed whenever
+      // memory is consumed or released via a MemTracker, so on a system with
+      // queries executing it will be refreshed frequently. However if the system
+      // is idle, we need to refresh the tracker occasionally since untracked
+      // memory may be allocated or freed, e.g. by background threads.
+      if (env->process_mem_tracker() != nullptr) {
+        env->process_mem_tracker()->RefreshConsumptionFromMetric();
+      }
 #endif
+    }
+    // Periodically refresh values of the aggregate memory metrics to ensure they are
+    // somewhat up-to-date.
+    AggregateMemoryMetrics::Refresh();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/runtime/exec-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index 6d5fb7a..960e3c9 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -241,7 +241,7 @@ Status ExecEnv::StartServices() {
 
   // Limit of -1 means no memory limit.
   mem_tracker_.reset(new MemTracker(
-      AggregateMemoryMetric::TOTAL_USED, bytes_limit > 0 ? bytes_limit : -1, "Process"));
+      AggregateMemoryMetrics::TOTAL_USED, bytes_limit > 0 ? bytes_limit : -1, "Process"));
   if (buffer_pool_ != nullptr) {
     // Add BufferPool MemTrackers for cached memory that is not tracked against queries
     // but is included in process memory consumption.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/util/default-path-handlers.cc
----------------------------------------------------------------------
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index 508ea45..17d2c94 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -31,6 +31,7 @@
 #include "service/impala-server.h"
 #include "util/common-metrics.h"
 #include "util/debug-util.h"
+#include "util/mem-info.h"
 #include "util/pprof-path-handlers.h"
 #include "util/mem-info.h"
 #include "util/cpu-info.h"
@@ -149,7 +150,17 @@ void MemUsageHandler(MemTracker* mem_tracker, MetricGroup* metric_group,
   Value detailed(mem_tracker->LogUsage().c_str(), document->GetAllocator());
   document->AddMember("detailed", detailed, document->GetAllocator());
 
+  Value systeminfo(MemInfo::DebugString().c_str(), document->GetAllocator());
+  document->AddMember("systeminfo", systeminfo, document->GetAllocator());
+
   if (metric_group != nullptr) {
+    MetricGroup* aggregate_group = metric_group->FindChildGroup("memory");
+    if (aggregate_group != nullptr) {
+      Value json_metrics(kObjectType);
+      aggregate_group->ToJson(false, document, &json_metrics);
+      document->AddMember(
+          "aggregate_metrics", json_metrics["metrics"], document->GetAllocator());
+    }
     MetricGroup* jvm_group = metric_group->FindChildGroup("jvm");
     if (jvm_group != nullptr) {
       Value jvm(kObjectType);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/util/mem-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/mem-info.cc b/be/src/util/mem-info.cc
index 89144b2..4d551f1 100644
--- a/be/src/util/mem-info.cc
+++ b/be/src/util/mem-info.cc
@@ -19,14 +19,16 @@
 #include "util/debug-util.h"
 #include "util/string-parser.h"
 
-#include <boost/algorithm/string.hpp>
-#include <boost/lexical_cast.hpp>
-#include <iostream>
-#include <fstream>
-#include <sstream>
 #include <stdlib.h>
 #include <string.h>
 #include <unistd.h>
+#include <cctype>
+#include <fstream>
+#include <iostream>
+#include <sstream>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/lexical_cast.hpp>
 
 #include "util/pretty-printer.h"
 
@@ -34,6 +36,7 @@
 
 using boost::algorithm::is_any_of;
 using boost::algorithm::split;
+using boost::algorithm::trim;
 using boost::algorithm::token_compress_on;
 
 namespace impala {
@@ -96,13 +99,98 @@ void MemInfo::ParseOvercommit() {
   overcommit_s >> vm_overcommit_;
 }
 
+bool MemInfo::HaveSmaps() {
+  MappedMemInfo result;
+  ifstream smaps("/proc/self/smaps", ios::in);
+  return smaps.good();
+}
+
+MappedMemInfo MemInfo::ParseSmaps() {
+  MappedMemInfo result;
+  ifstream smaps("/proc/self/smaps", ios::in);
+  if (!smaps) {
+    LOG_FIRST_N(INFO, 1) << "Could not open smaps";
+    return result;
+  }
+  while (smaps) {
+    string line;
+    getline(smaps, line);
+    if (line.empty()) continue;
+    if (isdigit(line[0])) {
+      // Line is the start of a new mapping, of form:
+      // 561ceff9c000-561ceffa1000 rw-p 00000000 00:00 0
+      ++result.num_maps;
+      continue;
+    }
+    // Line is in the form of <Name>: <value>, e.g.:
+    // Size: 1084 kB
+    // VmFlags: rd ex mr mw me dw
+    size_t colon_pos = line.find(':');
+    if (colon_pos == string::npos) continue;
+    string name = line.substr(0, colon_pos);
+    string value = line.substr(colon_pos + 1, string::npos);
+    trim(value);
+
+    // Use atol() to parse the value, ignoring " kB" suffix.
+    if (name == "Size") {
+      result.size_kb += atol(value.c_str());
+    } else if (name == "Rss") {
+      result.rss_kb += atol(value.c_str());
+    } else if (name == "AnonHugePages") {
+      result.anon_huge_pages_kb += atol(value.c_str());
+    }
+  }
+  return result;
+}
+
+ThpConfig MemInfo::ParseThpConfig() {
+  ThpConfig result;
+  result.enabled = GetThpConfigVal("enabled");
+  result.defrag = GetThpConfigVal("defrag");
+  result.khugepaged_defrag = GetThpConfigVal("khugepaged/defrag");
+  return result;
+}
+
+string MemInfo::GetThpConfigVal(const string& relative_path) {
+  // This is the standard location for the configs.
+  ifstream file("/sys/kernel/mm/transparent_hugepage/" + relative_path);
+  if (!file) {
+    // Some earlier versions of CentOS/RHEL put the configs in a different place.
+    file.open("/sys/kernel/mm/redhat_transparent_hugepage/" + relative_path);
+    if (!file) {
+      LOG_FIRST_N(INFO, 1) << "Could not open thp config: " << relative_path;
+      return "<unknown>";
+    }
+  }
+  string result;
+  getline(file, result);
+  return result;
+}
+
 string MemInfo::DebugString() {
   DCHECK(initialized_);
   stringstream stream;
-  stream << "Physical Memory: "
-         << PrettyPrinter::Print(physical_mem_, TUnit::BYTES)
+  stream << "Physical Memory: " << PrettyPrinter::Print(physical_mem_, TUnit::BYTES)
          << endl;
+  stream << ParseThpConfig().DebugString();
+  return stream.str();
+}
+
+string MappedMemInfo::DebugString() const {
+  stringstream stream;
+  stream << "Number of mappings: " << num_maps << endl;
+  stream << "Total mapping (kB): " << size_kb << endl;
+  stream << "RSS (kB): " << rss_kb << endl;
+  stream << "Anon huge pages (kB): " << anon_huge_pages_kb << endl;
   return stream.str();
 }
 
+string ThpConfig::DebugString() const {
+  stringstream stream;
+  stream << "Transparent Huge Pages Config:" << endl;
+  stream << "  enabled: " << enabled << endl;
+  stream << "  defrag: " << defrag << endl;
+  stream << "  khugepaged defrag: " << khugepaged_defrag << endl;
+  return stream.str();
+}
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/util/mem-info.h
----------------------------------------------------------------------
diff --git a/be/src/util/mem-info.h b/be/src/util/mem-info.h
index f7ef844..2e32d94 100644
--- a/be/src/util/mem-info.h
+++ b/be/src/util/mem-info.h
@@ -25,6 +25,37 @@
 
 namespace impala {
 
+/// Information obtained from /proc/<pid>/smaps.
+struct MappedMemInfo {
+  // Number of memory maps.
+  int64_t num_maps = 0;
+
+  // Total size of memory maps (i.e. virtual memory size) in kilobytes.
+  int64_t size_kb = 0;
+
+  // RSS in kilobytes
+  int64_t rss_kb = 0;
+
+  // Kilobytes of anonymous huge pages.
+  int64_t anon_huge_pages_kb = 0;
+
+  std::string DebugString() const;
+};
+
+/// Information about the system transparent huge pages config.
+struct ThpConfig {
+  // Whether THP is enabled. Just contains the raw string, e.g. "[always] madvise never".
+  std::string enabled;
+
+  // Whether synchronous THP defrag is enabled, e.g. "[always] madvise never".
+  std::string defrag;
+
+  // Whether THP defrag via khugepaged is enabled. Usually "0"/"1".
+  std::string khugepaged_defrag;
+
+  std::string DebugString() const;
+};
+
 /// Provides the amount of physical memory available.
 /// Populated from /proc/meminfo.
 /// TODO: Allow retrieving of cgroup memory limits,
@@ -54,12 +85,27 @@ class MemInfo {
     return commit_limit_;
   }
 
+  /// Return true if the /proc/<pid>/smaps file is present and can be opened.
+  static bool HaveSmaps();
+
+  /// Parse /proc/<pid>/smaps for this process and extract relevant information.
+  /// Logs a warning if the file could not be opened or had an unexpected format.
+  static MappedMemInfo ParseSmaps();
+
+  /// Parse the transparent huge pages configs.
+  /// Logs a warning if a file could not be opened or had an unexpected format.
+  static ThpConfig ParseThpConfig();
+
   static std::string DebugString();
 
  private:
-
   static void ParseOvercommit();
 
+  /// Get the config value from a file, trying the path relative to both
+  /// /sys/kernel/mm/transparent_hugepage and /sys/kernel/mm/redhat_transparent_hugepage.
+  /// Assumes the file has a single line only.
+  static std::string GetThpConfigVal(const std::string& relative_path);
+
   static bool initialized_;
   static int64_t physical_mem_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/util/memory-metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/memory-metrics.cc b/be/src/util/memory-metrics.cc
index b7aa3a6..f2ddcb9 100644
--- a/be/src/util/memory-metrics.cc
+++ b/be/src/util/memory-metrics.cc
@@ -23,6 +23,7 @@
 #include "runtime/bufferpool/buffer-pool.h"
 #include "runtime/bufferpool/reservation-tracker.h"
 #include "util/jni-util.h"
+#include "util/mem-info.h"
 #include "util/time.h"
 
 using boost::algorithm::to_lower;
@@ -31,13 +32,20 @@ using namespace strings;
 
 DECLARE_bool(mmap_buffers);
 
-SumGauge<uint64_t>* AggregateMemoryMetric::TOTAL_USED = nullptr;
+SumGauge<uint64_t>* AggregateMemoryMetrics::TOTAL_USED = nullptr;
+UIntGauge* AggregateMemoryMetrics::NUM_MAPS = nullptr;
+UIntGauge* AggregateMemoryMetrics::MAPPED_BYTES = nullptr;
+UIntGauge* AggregateMemoryMetrics::RSS = nullptr;
+UIntGauge* AggregateMemoryMetrics::ANON_HUGE_PAGE_BYTES = nullptr;
+StringProperty* AggregateMemoryMetrics::THP_ENABLED = nullptr;
+StringProperty* AggregateMemoryMetrics::THP_DEFRAG = nullptr;
+StringProperty* AggregateMemoryMetrics::THP_KHUGEPAGED_DEFRAG = nullptr;
 
-TcmallocMetric* TcmallocMetric::BYTES_IN_USE = NULL;
-TcmallocMetric* TcmallocMetric::PAGEHEAP_FREE_BYTES = NULL;
-TcmallocMetric* TcmallocMetric::TOTAL_BYTES_RESERVED = NULL;
-TcmallocMetric* TcmallocMetric::PAGEHEAP_UNMAPPED_BYTES = NULL;
-TcmallocMetric::PhysicalBytesMetric* TcmallocMetric::PHYSICAL_BYTES_RESERVED = NULL;
+TcmallocMetric* TcmallocMetric::BYTES_IN_USE = nullptr;
+TcmallocMetric* TcmallocMetric::PAGEHEAP_FREE_BYTES = nullptr;
+TcmallocMetric* TcmallocMetric::TOTAL_BYTES_RESERVED = nullptr;
+TcmallocMetric* TcmallocMetric::PAGEHEAP_UNMAPPED_BYTES = nullptr;
+TcmallocMetric::PhysicalBytesMetric* TcmallocMetric::PHYSICAL_BYTES_RESERVED = nullptr;
 
 AsanMallocMetric* AsanMallocMetric::BYTES_ALLOCATED = nullptr;
 
@@ -76,34 +84,71 @@ Status impala::RegisterMemoryMetrics(MetricGroup* metrics, bool register_jvm_met
       new AsanMallocMetric(MetricDefs::Get("asan-total-bytes-allocated")));
   used_metrics.push_back(AsanMallocMetric::BYTES_ALLOCATED);
 #else
+  MetricGroup* tcmalloc_metrics = metrics->GetOrCreateChildGroup("tcmalloc");
   // We rely on TCMalloc for our global memory metrics, so skip setting them up
   // if we're not using TCMalloc.
   TcmallocMetric::BYTES_IN_USE = TcmallocMetric::CreateAndRegister(
-      metrics, "tcmalloc.bytes-in-use", "generic.current_allocated_bytes");
+      tcmalloc_metrics, "tcmalloc.bytes-in-use", "generic.current_allocated_bytes");
 
   TcmallocMetric::TOTAL_BYTES_RESERVED = TcmallocMetric::CreateAndRegister(
-      metrics, "tcmalloc.total-bytes-reserved", "generic.heap_size");
+      tcmalloc_metrics, "tcmalloc.total-bytes-reserved", "generic.heap_size");
 
-  TcmallocMetric::PAGEHEAP_FREE_BYTES = TcmallocMetric::CreateAndRegister(metrics,
-      "tcmalloc.pageheap-free-bytes", "tcmalloc.pageheap_free_bytes");
+  TcmallocMetric::PAGEHEAP_FREE_BYTES = TcmallocMetric::CreateAndRegister(
+      tcmalloc_metrics, "tcmalloc.pageheap-free-bytes", "tcmalloc.pageheap_free_bytes");
 
-  TcmallocMetric::PAGEHEAP_UNMAPPED_BYTES = TcmallocMetric::CreateAndRegister(metrics,
-      "tcmalloc.pageheap-unmapped-bytes", "tcmalloc.pageheap_unmapped_bytes");
+  TcmallocMetric::PAGEHEAP_UNMAPPED_BYTES =
+      TcmallocMetric::CreateAndRegister(tcmalloc_metrics,
+          "tcmalloc.pageheap-unmapped-bytes", "tcmalloc.pageheap_unmapped_bytes");
 
   TcmallocMetric::PHYSICAL_BYTES_RESERVED =
-      metrics->RegisterMetric(new TcmallocMetric::PhysicalBytesMetric(
+      tcmalloc_metrics->RegisterMetric(new TcmallocMetric::PhysicalBytesMetric(
           MetricDefs::Get("tcmalloc.physical-bytes-reserved")));
 
   used_metrics.push_back(TcmallocMetric::PHYSICAL_BYTES_RESERVED);
 #endif
-  AggregateMemoryMetric::TOTAL_USED = metrics->RegisterMetric(
+  MetricGroup* aggregate_metrics = metrics->GetOrCreateChildGroup("memory");
+  AggregateMemoryMetrics::TOTAL_USED = aggregate_metrics->RegisterMetric(
       new SumGauge<uint64_t>(MetricDefs::Get("memory.total-used"), used_metrics));
   if (register_jvm_metrics) {
     RETURN_IF_ERROR(JvmMetric::InitMetrics(metrics->GetOrCreateChildGroup("jvm")));
   }
+
+  if (MemInfo::HaveSmaps()) {
+    AggregateMemoryMetrics::NUM_MAPS =
+        aggregate_metrics->AddGauge<uint64_t>("memory.num-maps", 0U);
+    AggregateMemoryMetrics::MAPPED_BYTES =
+        aggregate_metrics->AddGauge<uint64_t>("memory.mapped-bytes", 0U);
+    AggregateMemoryMetrics::RSS = aggregate_metrics->AddGauge<uint64_t>("memory.rss", 0U);
+    AggregateMemoryMetrics::ANON_HUGE_PAGE_BYTES =
+        aggregate_metrics->AddGauge<uint64_t>("memory.anon-huge-page-bytes", 0U);
+  }
+  ThpConfig thp_config = MemInfo::ParseThpConfig();
+  AggregateMemoryMetrics::THP_ENABLED =
+      aggregate_metrics->AddProperty("memory.thp.enabled", thp_config.enabled);
+  AggregateMemoryMetrics::THP_DEFRAG =
+      aggregate_metrics->AddProperty("memory.thp.defrag", thp_config.defrag);
+  AggregateMemoryMetrics::THP_KHUGEPAGED_DEFRAG = aggregate_metrics->AddProperty(
+      "memory.thp.khugepaged-defrag", thp_config.khugepaged_defrag);
+  AggregateMemoryMetrics::Refresh();
   return Status::OK();
 }
 
+void AggregateMemoryMetrics::Refresh() {
+  if (NUM_MAPS != nullptr) {
+    // Only call ParseSmaps() if the metrics were created.
+    MappedMemInfo map_info = MemInfo::ParseSmaps();
+    NUM_MAPS->set_value(map_info.num_maps);
+    MAPPED_BYTES->set_value(map_info.size_kb * 1024);
+    RSS->set_value(map_info.rss_kb * 1024);
+    ANON_HUGE_PAGE_BYTES->set_value(map_info.anon_huge_pages_kb * 1024);
+  }
+
+  ThpConfig thp_config = MemInfo::ParseThpConfig();
+  THP_ENABLED->set_value(thp_config.enabled);
+  THP_DEFRAG->set_value(thp_config.defrag);
+  THP_KHUGEPAGED_DEFRAG->set_value(thp_config.khugepaged_defrag);
+}
+
 JvmMetric* JvmMetric::CreateAndRegister(MetricGroup* metrics, const string& key,
     const string& pool_name, JvmMetric::JvmMetricType type) {
   string pool_name_for_key = pool_name;
@@ -120,7 +165,7 @@ JvmMetric::JvmMetric(const TMetricDef& def, const string& mempool_name,
 }
 
 Status JvmMetric::InitMetrics(MetricGroup* metrics) {
-  DCHECK(metrics != NULL);
+  DCHECK(metrics != nullptr);
   TGetJvmMetricsRequest request;
   request.get_all = true;
   TGetJvmMetricsResponse response;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/be/src/util/memory-metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/memory-metrics.h b/be/src/util/memory-metrics.h
index 5149d9c..ffc3d20 100644
--- a/be/src/util/memory-metrics.h
+++ b/be/src/util/memory-metrics.h
@@ -37,14 +37,45 @@ class ReservationTracker;
 class Thread;
 
 /// Memory metrics including TCMalloc and BufferPool memory.
-class AggregateMemoryMetric {
+class AggregateMemoryMetrics {
  public:
   /// The sum of Tcmalloc TOTAL_BYTES_RESERVED and BufferPool SYSTEM_ALLOCATED.
   /// Approximates the total amount of physical memory consumed by the backend (i.e. not
   /// including JVM memory), which is either in use by queries or cached by the BufferPool
-  /// or TcMalloc. NULL when running under ASAN.
+  /// or the malloc implementation.
   /// TODO: IMPALA-691 - consider changing this to include JVM memory.
   static SumGauge<uint64_t>* TOTAL_USED;
+
+  /// The total number of virtual memory regions for the process.
+  /// The value must be refreshed by calling Refresh().
+  static UIntGauge* NUM_MAPS;
+
+  /// The total size of virtual memory regions for the process.
+  /// The value must be refreshed by calling Refresh().
+  static UIntGauge* MAPPED_BYTES;
+
+  /// The total RSS of all virtual memory regions for the process.
+  /// The value must be refreshed by calling Refresh().
+  static UIntGauge* RSS;
+
+  /// The total RSS of all virtual memory regions for the process.
+  /// The value must be refreshed by calling Refresh().
+  static UIntGauge* ANON_HUGE_PAGE_BYTES;
+
+  /// The string reporting the /enabled setting for transparent huge pages.
+  /// The value must be refreshed by calling Refresh().
+  static StringProperty* THP_ENABLED;
+
+  /// The string reporting the /defrag setting for transparent huge pages.
+  /// The value must be refreshed by calling Refresh().
+  static StringProperty* THP_DEFRAG;
+
+  /// The string reporting the khugepaged/defrag setting for transparent huge pages.
+  /// The value must be refreshed by calling Refresh().
+  static StringProperty* THP_KHUGEPAGED_DEFRAG;
+
+  /// Refreshes values of any of the aggregate metrics that require refreshing.
+  static void Refresh();
 };
 
 /// Specialised metric which exposes numeric properties from tcmalloc.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/common/thrift/metrics.json
----------------------------------------------------------------------
diff --git a/common/thrift/metrics.json b/common/thrift/metrics.json
index 251ea14..2fbb6fc 100644
--- a/common/thrift/metrics.json
+++ b/common/thrift/metrics.json
@@ -1208,6 +1208,90 @@
     "key": "memory.total-used"
   },
   {
+    "description": "Total number of memory mappings in this process.",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Number of memory mappings.",
+    "units": "NONE",
+    "kind": "GAUGE",
+    "key": "memory.num-maps"
+  },
+  {
+    "description": "Total bytes of memory mappings in this process (the virtual memory size).",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Total Virtual Memory.",
+    "units": "BYTES",
+    "kind": "GAUGE",
+    "key": "memory.mapped-bytes"
+  },
+  {
+    "description": "Resident set size (RSS) of this process, including TCMalloc, buffer pool and Jvm.",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Resident set size.",
+    "units": "BYTES",
+    "kind": "GAUGE",
+    "key": "memory.rss"
+  },
+  {
+    "description": "Total bytes of anonymous (a.k.a. transparent) huge pages used by this process.",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Anonymous Huge Pages.",
+    "units": "BYTES",
+    "kind": "GAUGE",
+    "key": "memory.anon-huge-page-bytes"
+  },
+  {
+    "description": "The system-wide 'enabled' setting for Transparent Huge Pages.",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Transparent Huge Pages Enabled.",
+    "units": "NONE",
+    "kind": "PROPERTY",
+    "key": "memory.thp.enabled"
+  },
+  {
+    "description": "The system-wide 'defrag' setting for Transparent Huge Pages.",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Transparent Huge Pages Defrag Enabled.",
+    "units": "NONE",
+    "kind": "PROPERTY",
+    "key": "memory.thp.defrag"
+  },
+  {
+    "description": "The system-wide 'defrag' setting for khugepaged.",
+    "contexts": [
+      "STATESTORE",
+      "CATALOGSERVER",
+      "IMPALAD"
+    ],
+    "label": "Khugepaged Defrag Enabled.",
+    "units": "NONE",
+    "kind": "PROPERTY",
+    "key": "memory.thp.khugepaged-defrag"
+  },
+  {
     "description": "The number of running threads in this process.",
     "contexts": [
       "STATESTORE",

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3f82d157/www/memz.tmpl
----------------------------------------------------------------------
diff --git a/www/memz.tmpl b/www/memz.tmpl
index fca45c0..9c629d8 100644
--- a/www/memz.tmpl
+++ b/www/memz.tmpl
@@ -28,6 +28,41 @@ Memory consumption / limit: <strong>{{consumption}}</strong> / <strong>{{mem_lim
 <h3>tcmalloc</h3>
 <pre>{{overview}}</pre>
 
+<h3>System</h3>
+<pre>{{systeminfo}}</pre>
+
+{{?aggregate_metrics}}
+<h3>Process and system memory metrics</h3>
+<table class='table table-bordered table-hover'>
+  <tr>
+    <th>Name</th>
+    <th>Value</th>
+    <th>Description</th>
+  </tr>
+{{/aggregate_metrics}}
+{{#aggregate_metrics}}
+  <tr>
+    <td><tt>{{name}}</tt></td>
+    {{! Is this a stats metric? }}
+    {{?mean}}
+    <td>
+      Last (of {{count}}): <strong>{{last}}</strong>.
+      Min: {{min}}, max: {{max}}, avg: {{mean}}</td>
+    {{/mean}}
+    {{^mean}}
+    <td>
+      {{human_readable}}
+    </td>
+    {{/mean}}
+    <td>
+      {{description}}
+    </td>
+  </tr>
+{{/aggregate_metrics}}
+{{?aggregate_metrics}}
+</table>
+{{/aggregate_metrics}}
+
 {{?buffer_pool}}
 <h3>Buffer pool memory metrics</h3>
 <table class='table table-bordered table-hover'>