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 2016/08/22 07:37:03 UTC

[4/4] kudu git commit: KUDU-1571. Generate "deprecated" metrics in MDL

KUDU-1571. Generate "deprecated" metrics in MDL

This checks in the "--dump_metrics_json" output from already-released
versions, and changes the generate_mdl.py script so that any metrics
which were ever present in an earlier version are carried through to the
current version of the MDL, but marked as deprecated.

This works around an issue in which Cloudera Manager doesn't properly
handle the removal of metrics. Namely, if a dashboard (user-generated or
built-in) references a removed metric, it will produce an NPE.

This will now become part of the release process: after a release, we
should check in the output of 'dump_metrics_json' so that the metrics in
that release will be carried forward.

I verified the results using both the newly included
'check_csd_compatibility' script as well as installing a newly built
0.10.x CSD on a cluster. Whereas the tablet server status page was
previously broken, now it shows the remote bootstrap metrics as
DEPRECATED.

Change-Id: I121203d781ab593cb94d7248d164cdad7bf26e78
Reviewed-on: http://gerrit.cloudera.org:8080/4067
Tested-by: Kudu Jenkins
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/062996ad
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/062996ad
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/062996ad

Branch: refs/heads/master
Commit: 062996ad7d1c3aa90cd9ea5ec11b73dc17e085d6
Parents: b3371ab
Author: Todd Lipcon <to...@apache.org>
Authored: Fri Aug 19 17:43:04 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Aug 22 07:36:37 2016 +0000

----------------------------------------------------------------------
 build-support/release/rat_exclude_files.txt     |    1 +
 java/kudu-csd/check_csd_compatibility.py        |   92 ++
 java/kudu-csd/generate_mdl.py                   |   83 +-
 .../old-version-metrics/0.10.0-master.json      | 1236 ++++++++++++++++++
 .../old-version-metrics/0.10.0-tserver.json     | 1124 ++++++++++++++++
 .../old-version-metrics/0.9.1-master.json       | 1196 +++++++++++++++++
 .../old-version-metrics/0.9.1-tserver.json      | 1116 ++++++++++++++++
 7 files changed, 4831 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/062996ad/build-support/release/rat_exclude_files.txt
----------------------------------------------------------------------
diff --git a/build-support/release/rat_exclude_files.txt b/build-support/release/rat_exclude_files.txt
index 630f414..72016ce 100644
--- a/build-support/release/rat_exclude_files.txt
+++ b/build-support/release/rat_exclude_files.txt
@@ -9,6 +9,7 @@ pax_global_header
 *.pdf
 version.txt
 build-support/release/rat_exclude_files.txt
+java/kudu-csd/old-version-metrics/*.json
 java/kudu-client/src/main/java/com/google/protobuf/ZeroCopyLiteralByteString.java
 java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduClient.java
 java/kudu-client/src/main/java/org/apache/kudu/client/AsyncKuduScanner.java

http://git-wip-us.apache.org/repos/asf/kudu/blob/062996ad/java/kudu-csd/check_csd_compatibility.py
----------------------------------------------------------------------
diff --git a/java/kudu-csd/check_csd_compatibility.py b/java/kudu-csd/check_csd_compatibility.py
new file mode 100755
index 0000000..46c561c
--- /dev/null
+++ b/java/kudu-csd/check_csd_compatibility.py
@@ -0,0 +1,92 @@
+#!/usr/bin/env python
+#
+# 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.
+
+# This tool checks for compatibility between two CSD jars.
+# Namely, it ensures that no metrics have been removed.
+
+import argparse
+import json
+import sys
+import zipfile
+
+# The path within a CSD that contains the MDL file.
+MDL_PATH = "descriptor/service.mdl"
+
+
+def metrics_from_mdl(mdl):
+  """
+  Return a set() representing the metrics in the given MDL JSON object.
+
+  Each entry is a tuple of either:
+    ('entity', entity_name, metric_name)
+  or
+    ('role', role_name, metric_name)
+  """
+  metrics = set()
+  for entity_def in mdl['metricEntityTypeDefinitions']:
+    for metric_def in entity_def['metricDefinitions']:
+      metrics.add(('entity', entity_def['name'], metric_def['name']))
+  for role_def in mdl['roles']:
+    for metric_def in role_def['metricDefinitions']:
+      metrics.add(('role', role_def['name'], metric_def['name']))
+  return metrics
+
+
+def check_mdl_compat(old_zip, new_zip, args):
+  old_mdl = json.load(old_zip.open(MDL_PATH))
+  new_mdl = json.load(new_zip.open(MDL_PATH))
+
+  old_metrics = metrics_from_mdl(old_mdl)
+  new_metrics = metrics_from_mdl(new_mdl)
+
+  added_metrics = new_metrics.difference(old_metrics)
+  removed_metrics = old_metrics.difference(new_metrics)
+
+  print "Added %d metric(s):" % len(added_metrics)
+  for m_type, m_entity, m_name in added_metrics:
+    print "  %s metric %s" % (m_entity, m_name)
+  if len(removed_metrics):
+    print "Removed %d metric(s):" % len(removed_metrics)
+    for m_type, m_entity, m_name in removed_metrics:
+      print "  %s metric %s" % (m_entity, m_name)
+    print "Compatibility check FAILED"
+    sys.exit(1)
+  print "No metrics were removed."
+  print "Compatibility check PASSED"
+
+
+def main():
+  p = argparse.ArgumentParser(
+    description=("Checks for compatibility between CSD JARs. " +
+                 "May also generate a file containing JSON for " +
+                 "metrics that were removed between the old version "
+                 "of the CSD and the specified new one."))
+  p.add_argument("old_jar", metavar="KUDU-old.jar", type=str,
+                 help="The old CSD JAR to compare")
+  p.add_argument("new_jar", metavar="KUDU-new.jar", type=str,
+                 help="The new CSD JAR to compare")
+  args = p.parse_args()
+
+  old_zip = zipfile.ZipFile(args.old_jar)
+  new_zip = zipfile.ZipFile(args.new_jar)
+  check_mdl_compat(old_zip, new_zip, args)
+
+
+if __name__ == "__main__":
+  main()

http://git-wip-us.apache.org/repos/asf/kudu/blob/062996ad/java/kudu-csd/generate_mdl.py
----------------------------------------------------------------------
diff --git a/java/kudu-csd/generate_mdl.py b/java/kudu-csd/generate_mdl.py
index 2d34aa1..f2c562a 100755
--- a/java/kudu-csd/generate_mdl.py
+++ b/java/kudu-csd/generate_mdl.py
@@ -27,6 +27,7 @@
 # argument.
 
 import collections
+from distutils.version import LooseVersion
 try:
   import simplejson as json
 except:
@@ -37,17 +38,20 @@ import sys
 
 BINARIES=["kudu-master", "kudu-tserver"]
 
+BASE_DIR = os.path.dirname(os.path.abspath(__file__))
 RELATIVE_BUILD_DIR="../../build/latest/bin"
 
+DEPRECATION_MESSAGE = " This metric is no longer produced in " + \
+                      "current versions of Kudu."
+
 def find_binary(bin_name):
-  dirname, _ = os.path.split(os.path.abspath(__file__))
-  build_dir = os.path.join(dirname, RELATIVE_BUILD_DIR)
+  build_dir = os.path.join(BASE_DIR, RELATIVE_BUILD_DIR)
   path = os.path.join(build_dir, bin_name)
   if os.path.exists(path):
     return path
   raise Exception("Cannot find %s in build dir %s" % (bin_name, build_dir))
 
-def load_all_metrics():
+def load_current_metrics():
   """
   For each binary, dump and parse its metrics schema by running it with
   the --dump_metrics_json flag.
@@ -67,9 +71,33 @@ def load_all_metrics():
       sys.exit(1)
 
     metrics_dump = json.loads(stdout)
-    all_metrics.extend(m for m in metrics_dump['metrics'])
+    all_metrics.extend(metrics_dump['metrics'])
   return all_metrics
 
+
+def load_historical_metrics():
+  """
+  Load the metrics dump output from older versions of Kudu. These are checked
+  into the source repository in the "old-version-metrics" directory.
+
+  If any of these metrics has been removed in the current version of Kudu,
+  it will be included, but marked as deprecated and no longer produced.
+  """
+
+  old_metrics = {}
+  old_dir = os.path.join(BASE_DIR, "old-version-metrics")
+
+  # Load the metrics dumps from previous versions in ascending
+  # version order. This way, we retain the latest description/name
+  # of the metric in the remaining (deprecated) metric.
+  for path in sorted(os.listdir(old_dir), key=LooseVersion):
+    if not path.endswith(".json") or path.startswith("."):
+      continue
+    j = json.load(file(os.path.join(old_dir, path)))
+    old_metrics.update((m['name'], m) for m in j['metrics'])
+  return old_metrics.values()
+
+
 def append_sentence(a, b):
   if not a.endswith("."):
     a += "."
@@ -104,29 +132,51 @@ def metric_to_mdl_entries(m):
     description=m['description'],
     label=m['label'])]
 
-def metrics_to_mdl(metrics):
+
+def deprecate_entries(mdl_entries):
+  """ For each entry in 'mdl_entries', mark it as deprecated. """
+  for m in mdl_entries:
+    m['label'] = "DEPRECATED: %s" % m['label']
+    m['description'] += DEPRECATION_MESSAGE
+
+
+def metrics_to_mdl(metrics, historical_metrics):
   """
   For each metric returned by the daemon, convert it to the MDL-compatible dictionary.
+  Metrics which are in 'historical_metrics' but not in 'metrics' are added, but marked
+  as deprecated.
+
   Returns a map of entity_type_name -> [metric dicts].
   """
   seen = set()
 
   by_entity = collections.defaultdict(lambda: [])
-  for m in metrics:
-    # Don't process any metric more than once. Some metrics show up
-    # in both daemons.
-    key = (m['entity_type'], m['name'])
-    if key in seen:
-      continue
-    seen.add(key)
+  def add_metrics(metrics, deprecated=False):
+    for m in metrics:
+      # Don't process any metric more than once. Some metrics show up
+      # in both daemons.
+      key = (m['entity_type'], m['name'])
+      if key in seen:
+        continue
+      seen.add(key)
+      # Convert to the format that CM expects.
+      mdl_entries = metric_to_mdl_entries(m)
+      if deprecated:
+        deprecate_entries(mdl_entries)
+      by_entity[m['entity_type']].extend(mdl_entries)
 
-    # Convert to the format that CM expects.
-    by_entity[m['entity_type']].extend(metric_to_mdl_entries(m))
+  # First add all the current metrics.
+  add_metrics(metrics)
+  # Then add any metrics from old versions that were not already
+  # added by the current version. These are marked as DEPRECATED.
+  add_metrics(historical_metrics, deprecated=True)
   return by_entity
 
+
 def main():
-  all_metrics = load_all_metrics()
-  metrics_by_entity = metrics_to_mdl(all_metrics)
+  all_metrics = load_current_metrics()
+  historical_metrics = load_historical_metrics()
+  metrics_by_entity = metrics_to_mdl(all_metrics, historical_metrics)
   server_metrics = metrics_by_entity['server']
   tablet_metrics = metrics_by_entity['tablet']
 
@@ -202,7 +252,6 @@ def main():
            metricDefinitions=tablet_metrics),
       ])
 
-  
   f = sys.stdout
   if len(sys.argv) > 1:
     f = open(sys.argv[1], 'w')

http://git-wip-us.apache.org/repos/asf/kudu/blob/062996ad/java/kudu-csd/old-version-metrics/0.10.0-master.json
----------------------------------------------------------------------
diff --git a/java/kudu-csd/old-version-metrics/0.10.0-master.json b/java/kudu-csd/old-version-metrics/0.10.0-master.json
new file mode 100644
index 0000000..5ade70f
--- /dev/null
+++ b/java/kudu-csd/old-version-metrics/0.10.0-master.json
@@ -0,0 +1,1236 @@
+{
+    "metrics": [
+        {
+            "name": "op_apply_queue_length",
+            "label": "Operation Apply Queue Length",
+            "type": "histogram",
+            "unit": "tasks",
+            "description": "Number of operations waiting to be applied to the tablet. High queue lengths indicate that the server is unable to process operations as fast as they are being written to the WAL.",
+            "entity_type": "server"
+        },
+        {
+            "name": "op_apply_queue_time",
+            "label": "Operation Apply Queue Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Time that operations spent waiting in the apply queue before being processed. High queue times indicate that the server is unable to process operations as fast as they are being written to the WAL.",
+            "entity_type": "server"
+        },
+        {
+            "name": "op_apply_run_time",
+            "label": "Operation Apply Run Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Time that operations spent being applied to the tablet. High values may indicate that the server is under-provisioned or that operations consist of very large batches.",
+            "entity_type": "server"
+        },
+        {
+            "name": "active_scanners",
+            "label": "Active Scanners",
+            "type": "gauge",
+            "unit": "scanners",
+            "description": "Number of scanners that are currently active",
+            "entity_type": "server"
+        },
+        {
+            "name": "scanners_expired",
+            "label": "Scanners Expired",
+            "type": "counter",
+            "unit": "scanners",
+            "description": "Number of scanners that have expired since service start",
+            "entity_type": "server"
+        },
+        {
+            "name": "scanner_duration",
+            "label": "Scanner Duration",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Histogram of the duration of active scanners on this tablet.",
+            "entity_type": "server"
+        },
+        {
+            "name": "memrowset_size",
+            "label": "MemRowSet Memory Usage",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Size of this tablet's memrowset",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "on_disk_size",
+            "label": "Tablet Size On Disk",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Size of this tablet on disk.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "rows_inserted",
+            "label": "Rows Inserted",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of rows inserted into this tablet since service start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "rows_upserted",
+            "label": "Rows Upserted",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of rows upserted into this tablet since service start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "rows_updated",
+            "label": "Rows Updated",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of row update operations performed on this tablet since service start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "rows_deleted",
+            "label": "Rows Deleted",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of row delete operations performed on this tablet since service start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scanner_rows_returned",
+            "label": "Scanner Rows Returned",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of rows returned by scanners to clients. This count is measured after predicates are applied, and thus is not a reflection of the amount of work being done by scanners.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scanner_cells_returned",
+            "label": "Scanner Cells Returned",
+            "type": "counter",
+            "unit": "cells",
+            "description": "Number of table cells returned by scanners to clients. This count is measured after predicates are applied, and thus is not a reflection of the amount of work being done by scanners.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scanner_bytes_returned",
+            "label": "Scanner Bytes Returned",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Number of bytes returned by scanners to clients. This count is measured after predicates are applied and the data is decoded for consumption by clients, and thus is not a reflection of the amount of work being done by scanners.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scanner_rows_scanned",
+            "label": "Scanner Rows Scanned",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of rows processed by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual table rows that have been processed by scan operations compared to the Scanner Rows Returned metric.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scanner_cells_scanned_from_disk",
+            "label": "Scanner Cells Scanned From Disk",
+            "type": "counter",
+            "unit": "cells",
+            "description": "Number of table cells processed by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual table cells that have been processed by scan operations compared to the Scanner Cells Returned metric.\nNote that this only counts data that has been flushed to disk, and does not include data read from in-memory stores. However, itincludes both cache misses and cache hits.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scanner_bytes_scanned_from_disk",
+            "label": "Scanner Bytes Scanned From Disk",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Number of bytes read by scan requests. This is measured as a raw count prior to application of predicates, deleted data,or MVCC-based filtering. Thus, this is a better measure of actual IO that has been caused by scan operations compared to the Scanner Bytes Returned metric.\nNote that this only counts data that has been flushed to disk, and does not include data read from in-memory stores. However, itincludes both cache misses and cache hits.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "insertions_failed_dup_key",
+            "label": "Duplicate Key Inserts",
+            "type": "counter",
+            "unit": "rows",
+            "description": "Number of inserts which failed because the key already existed",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "scans_started",
+            "label": "Scans Started",
+            "type": "counter",
+            "unit": "scanners",
+            "description": "Number of scanners which have been started on this tablet",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "bloom_lookups",
+            "label": "Bloom Filter Lookups",
+            "type": "counter",
+            "unit": "probes",
+            "description": "Number of times a bloom filter was consulted",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "key_file_lookups",
+            "label": "Key File Lookups",
+            "type": "counter",
+            "unit": "probes",
+            "description": "Number of times a key cfile was consulted",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "delta_file_lookups",
+            "label": "Delta File Lookups",
+            "type": "counter",
+            "unit": "probes",
+            "description": "Number of times a delta file was consulted",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "mrs_lookups",
+            "label": "MemRowSet Lookups",
+            "type": "counter",
+            "unit": "probes",
+            "description": "Number of times a MemRowSet was consulted.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "bytes_flushed",
+            "label": "Bytes Flushed",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Amount of data that has been flushed to disk by this tablet.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "bloom_lookups_per_op",
+            "label": "Bloom Lookups per Operation",
+            "type": "histogram",
+            "unit": "probes",
+            "description": "Tracks the number of bloom filter lookups performed by each operation. A single operation may perform several bloom filter lookups if the tablet is not fully compacted. High frequency of high values may indicate that compaction is falling behind.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "key_file_lookups_per_op",
+            "label": "Key Lookups per Operation",
+            "type": "histogram",
+            "unit": "probes",
+            "description": "Tracks the number of key file lookups performed by each operation. A single operation may perform several key file lookups if the tablet is not fully compacted and if bloom filters are not effectively culling lookups.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "delta_file_lookups_per_op",
+            "label": "Delta File Lookups per Operation",
+            "type": "histogram",
+            "unit": "probes",
+            "description": "Tracks the number of delta file lookups performed by each operation. A single operation may perform several delta file lookups if the tablet is not fully compacted. High frequency of high values may indicate that compaction is falling behind.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "write_op_duration_client_propagated_consistency",
+            "label": "Write Op Duration with Propagated Consistency",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Duration of writes to this tablet with external consistency set to CLIENT_PROPAGATED.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "write_op_duration_commit_wait_consistency",
+            "label": "Write Op Duration with Commit-Wait Consistency",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Duration of writes to this tablet with external consistency set to COMMIT_WAIT.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "commit_wait_duration",
+            "label": "Commit-Wait Duration",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Time spent waiting for COMMIT_WAIT external consistency writes for this tablet.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "snapshot_read_inflight_wait_duration",
+            "label": "Time Waiting For Snapshot Reads",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Time spent waiting for in-flight writes to complete for READ_AT_SNAPSHOT scans.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "flush_dms_running",
+            "label": "DeltaMemStore Flushes Running",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of delta memstore flushes currently running.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "flush_mrs_running",
+            "label": "MemRowSet Flushes Running",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of MemRowSet flushes currently running.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "compact_rs_running",
+            "label": "RowSet Compactions Running",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of RowSet compactions currently running.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "delta_minor_compact_rs_running",
+            "label": "Minor Delta Compactions Running",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of delta minor compactions currently running.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "delta_major_compact_rs_running",
+            "label": "Major Delta Compactions Running",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of delta major compactions currently running.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "flush_dms_duration",
+            "label": "DeltaMemStore Flush Duration",
+            "type": "histogram",
+            "unit": "milliseconds",
+            "description": "Time spent flushing DeltaMemStores.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "flush_mrs_duration",
+            "label": "MemRowSet Flush Duration",
+            "type": "histogram",
+            "unit": "milliseconds",
+            "description": "Time spent flushing MemRowSets.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "compact_rs_duration",
+            "label": "RowSet Compaction Duration",
+            "type": "histogram",
+            "unit": "milliseconds",
+            "description": "Time spent compacting RowSets.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "delta_minor_compact_rs_duration",
+            "label": "Minor Delta Compaction Duration",
+            "type": "histogram",
+            "unit": "milliseconds",
+            "description": "Time spent minor delta compacting.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "delta_major_compact_rs_duration",
+            "label": "Major Delta Compaction Duration",
+            "type": "histogram",
+            "unit": "seconds",
+            "description": "Seconds spent major delta compacting.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "leader_memory_pressure_rejections",
+            "label": "Leader Memory Pressure Rejections",
+            "type": "counter",
+            "unit": "requests",
+            "description": "Number of RPC requests rejected due to memory pressure while LEADER.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "op_prepare_queue_length",
+            "label": "Operation Prepare Queue Length",
+            "type": "histogram",
+            "unit": "tasks",
+            "description": "Number of operations waiting to be prepared within this tablet. High queue lengths indicate that the server is unable to process operations as fast as they are being written to the WAL.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "op_prepare_queue_time",
+            "label": "Operation Prepare Queue Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Time that operations spent waiting in the prepare queue before being processed. High queue times indicate that the server is unable to process operations as fast as they are being written to the WAL.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "op_prepare_run_time",
+            "label": "Operation Prepare Run Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Time that operations spent being prepared in the tablet. High values may indicate that the server is under-provisioned or that operations are experiencing high contention with one another for locks.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "all_transactions_inflight",
+            "label": "Transactions In Flight",
+            "type": "gauge",
+            "unit": "transactions",
+            "description": "Number of transactions currently in-flight, including any type.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "write_transactions_inflight",
+            "label": "Write Transactions In Flight",
+            "type": "gauge",
+            "unit": "transactions",
+            "description": "Number of write transactions currently in-flight",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "alter_schema_transactions_inflight",
+            "label": "Alter Schema Transactions In Flight",
+            "type": "gauge",
+            "unit": "transactions",
+            "description": "Number of alter schema transactions currently in-flight",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "transaction_memory_pressure_rejections",
+            "label": "Transaction Memory Pressure Rejections",
+            "type": "counter",
+            "unit": "transactions",
+            "description": "Number of transactions rejected because the tablet's transaction memory limit was reached.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_gc_running",
+            "label": "Log GCs Running",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of log GC operations currently running.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_gc_duration",
+            "label": "Log GC Duration",
+            "type": "histogram",
+            "unit": "milliseconds",
+            "description": "Time spent garbage collecting the logs.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "generic_current_allocated_bytes",
+            "label": "Heap Memory Usage",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Number of bytes used by the application. This will not typically match the memory use reported by the OS, because it does not include TCMalloc overhead or memory fragmentation.",
+            "entity_type": "server"
+        },
+        {
+            "name": "generic_heap_size",
+            "label": "Reserved Heap Memory",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Bytes of system memory reserved by TCMalloc.",
+            "entity_type": "server"
+        },
+        {
+            "name": "tcmalloc_pageheap_free_bytes",
+            "label": "Free Heap Memory",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Number of bytes in free, mapped pages in page heap. These bytes can be used to fulfill allocation requests. They always count towards virtual memory usage, and unless the underlying memory is swapped out by the OS, they also count towards physical memory usage.",
+            "entity_type": "server"
+        },
+        {
+            "name": "tcmalloc_pageheap_unmapped_bytes",
+            "label": "Unmapped Heap Memory",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Number of bytes in free, unmapped pages in page heap. These are bytes that have been released back to the OS, possibly by one of the MallocExtension \"Release\" calls. They can be used to fulfill allocation requests, but typically incur a page fault. They always count towards virtual memory usage, and depending on the OS, typically do not count towards physical memory usage.",
+            "entity_type": "server"
+        },
+        {
+            "name": "tcmalloc_max_total_thread_cache_bytes",
+            "label": "Thread Cache Memory Limit",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "A limit to how much memory TCMalloc dedicates for small objects. Higher numbers trade off more memory use for -- in some situations -- improved efficiency.",
+            "entity_type": "server"
+        },
+        {
+            "name": "tcmalloc_current_total_thread_cache_bytes",
+            "label": "Thread Cache Memory Usage",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "A measure of some of the memory TCMalloc is using (for small objects).",
+            "entity_type": "server"
+        },
+        {
+            "name": "glog_info_messages",
+            "label": "INFO-level Log Messages",
+            "type": "counter",
+            "unit": "messages",
+            "description": "Number of INFO-level log messages emitted by the application.",
+            "entity_type": "server"
+        },
+        {
+            "name": "glog_warning_messages",
+            "label": "WARNING-level Log Messages",
+            "type": "counter",
+            "unit": "messages",
+            "description": "Number of WARNING-level log messages emitted by the application.",
+            "entity_type": "server"
+        },
+        {
+            "name": "glog_error_messages",
+            "label": "ERROR-level Log Messages",
+            "type": "counter",
+            "unit": "messages",
+            "description": "Number of ERROR-level log messages emitted by the application.",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_server_GenericService_SetFlag",
+            "label": "kudu.server.GenericService.SetFlag RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.server.GenericService.SetFlag() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_server_GenericService_FlushCoverage",
+            "label": "kudu.server.GenericService.FlushCoverage RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.server.GenericService.FlushCoverage() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_server_GenericService_ServerClock",
+            "label": "kudu.server.GenericService.ServerClock RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.server.GenericService.ServerClock() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_server_GenericService_SetServerWallClockForTests",
+            "label": "kudu.server.GenericService.SetServerWallClockForTests RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.server.GenericService.SetServerWallClockForTests() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_server_GenericService_GetStatus",
+            "label": "kudu.server.GenericService.GetStatus RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.server.GenericService.GetStatus() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "majority_done_ops",
+            "label": "Leader Operations Acked by Majority",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of operations in the leader queue ack'd by a majority but not all peers.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "in_progress_ops",
+            "label": "Leader Operations in Progress",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of operations in the leader queue ack'd by a minority of peers.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_cache_num_ops",
+            "label": "Log Cache Operation Count",
+            "type": "gauge",
+            "unit": "operations",
+            "description": "Number of operations in the log cache.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_cache_size",
+            "label": "Log Cache Memory Usage",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Amount of memory in use for caching the local log.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "follower_memory_pressure_rejections",
+            "label": "Follower Memory Pressure Rejections",
+            "type": "counter",
+            "unit": "requests",
+            "description": "Number of RPC requests rejected due to memory pressure while FOLLOWER.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "raft_term",
+            "label": "Current Raft Consensus Term",
+            "type": "gauge",
+            "unit": "units",
+            "description": "Current Term of the Raft Consensus algorithm. This number increments each time a leader election is started.",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_reader_bytes_read",
+            "label": "Bytes Read From Log",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Data read from the WAL since tablet start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_reader_entries_read",
+            "label": "Entries Read From Log",
+            "type": "counter",
+            "unit": "entries",
+            "description": "Number of entries read from the WAL since tablet start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_reader_read_batch_latency",
+            "label": "Log Read Latency",
+            "type": "histogram",
+            "unit": "bytes",
+            "description": "Microseconds spent reading log entry batches",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_bytes_logged",
+            "label": "Bytes Written to WAL",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Number of bytes logged since service start",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_sync_latency",
+            "label": "Log Sync Latency",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent on synchronizing the log segment file",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_append_latency",
+            "label": "Log Append Latency",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent on appending to the log segment file",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_group_commit_latency",
+            "label": "Log Group Commit Latency",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent on committing an entire group",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_roll_latency",
+            "label": "Log Roll Latency",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent on rolling over to a new log segment file",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "log_entry_batches_per_group",
+            "label": "Log Group Commit Batch Size",
+            "type": "histogram",
+            "unit": "requests",
+            "description": "Number of log entry batches in a group commit group",
+            "entity_type": "tablet"
+        },
+        {
+            "name": "hybrid_clock_timestamp",
+            "label": "Hybrid Clock Timestamp",
+            "type": "gauge",
+            "unit": "microseconds",
+            "description": "Hybrid clock timestamp.",
+            "entity_type": "server"
+        },
+        {
+            "name": "hybrid_clock_error",
+            "label": "Hybrid Clock Error",
+            "type": "gauge",
+            "unit": "microseconds",
+            "description": "Server clock maximum error.",
+            "entity_type": "server"
+        },
+        {
+            "name": "logical_clock_timestamp",
+            "label": "Logical Clock Timestamp",
+            "type": "gauge",
+            "unit": "units",
+            "description": "Logical clock timestamp.",
+            "entity_type": "server"
+        },
+        {
+            "name": "code_cache_hits",
+            "label": "Codegen Cache Hits",
+            "type": "counter",
+            "unit": "hits",
+            "description": "Number of codegen cache hits since start",
+            "entity_type": "server"
+        },
+        {
+            "name": "code_cache_queries",
+            "label": "Codegen Cache Queries",
+            "type": "counter",
+            "unit": "queries",
+            "description": "Number of codegen cache queries (hits + misses) since start",
+            "entity_type": "server"
+        },
+        {
+            "name": "log_block_manager_bytes_under_management",
+            "label": "Bytes Under Management",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Number of bytes of data blocks currently under management",
+            "entity_type": "server"
+        },
+        {
+            "name": "log_block_manager_blocks_under_management",
+            "label": "Blocks Under Management",
+            "type": "gauge",
+            "unit": "blocks",
+            "description": "Number of data blocks currently under management",
+            "entity_type": "server"
+        },
+        {
+            "name": "log_block_manager_containers",
+            "label": "Number of Block Containers",
+            "type": "counter",
+            "unit": "log block containers",
+            "description": "Number of log block containers",
+            "entity_type": "server"
+        },
+        {
+            "name": "log_block_manager_full_containers",
+            "label": "Number of Full Block Counters",
+            "type": "counter",
+            "unit": "log block containers",
+            "description": "Number of full log block containers",
+            "entity_type": "server"
+        },
+        {
+            "name": "log_block_manager_unavailable_containers",
+            "label": "Number of Unavailable Log Block Containers",
+            "type": "counter",
+            "unit": "log block containers",
+            "description": "Number of non-full log block containers that are under root paths whose disks are full",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_manager_blocks_open_reading",
+            "label": "Data Blocks Open For Read",
+            "type": "gauge",
+            "unit": "blocks",
+            "description": "Number of data blocks currently open for reading",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_manager_blocks_open_writing",
+            "label": "Data Blocks Open For Write",
+            "type": "gauge",
+            "unit": "blocks",
+            "description": "Number of data blocks currently open for writing",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_manager_total_writable_blocks",
+            "label": "Data Blocks Opened For Write",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of data blocks opened for writing since service start",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_manager_total_readable_blocks",
+            "label": "Data Blocks Opened For Read",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of data blocks opened for reading since service start",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_manager_total_bytes_written",
+            "label": "Block Data Bytes Written",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Number of bytes of block data written since service start",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_manager_total_bytes_read",
+            "label": "Block Data Bytes Read",
+            "type": "counter",
+            "unit": "bytes",
+            "description": "Number of bytes of block data read since service start",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerService_Ping",
+            "label": "kudu.tserver.TabletServerService.Ping RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerService.Ping() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerService_Write",
+            "label": "kudu.tserver.TabletServerService.Write RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerService.Write() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerService_Scan",
+            "label": "kudu.tserver.TabletServerService.Scan RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerService.Scan() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerService_ScannerKeepAlive",
+            "label": "kudu.tserver.TabletServerService.ScannerKeepAlive RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerService.ScannerKeepAlive() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerService_ListTablets",
+            "label": "kudu.tserver.TabletServerService.ListTablets RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerService.ListTablets() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerService_Checksum",
+            "label": "kudu.tserver.TabletServerService.Checksum RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerService.Checksum() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletCopyService_BeginTabletCopySession",
+            "label": "kudu.tserver.TabletCopyService.BeginTabletCopySession RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletCopyService.BeginTabletCopySession() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletCopyService_CheckSessionActive",
+            "label": "kudu.tserver.TabletCopyService.CheckSessionActive RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletCopyService.CheckSessionActive() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletCopyService_FetchData",
+            "label": "kudu.tserver.TabletCopyService.FetchData RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletCopyService.FetchData() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletCopyService_EndTabletCopySession",
+            "label": "kudu.tserver.TabletCopyService.EndTabletCopySession RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletCopyService.EndTabletCopySession() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_UpdateConsensus",
+            "label": "kudu.consensus.ConsensusService.UpdateConsensus RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.UpdateConsensus() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_RequestConsensusVote",
+            "label": "kudu.consensus.ConsensusService.RequestConsensusVote RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.RequestConsensusVote() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_ChangeConfig",
+            "label": "kudu.consensus.ConsensusService.ChangeConfig RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.ChangeConfig() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_GetNodeInstance",
+            "label": "kudu.consensus.ConsensusService.GetNodeInstance RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.GetNodeInstance() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_RunLeaderElection",
+            "label": "kudu.consensus.ConsensusService.RunLeaderElection RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.RunLeaderElection() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_LeaderStepDown",
+            "label": "kudu.consensus.ConsensusService.LeaderStepDown RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.LeaderStepDown() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_GetLastOpId",
+            "label": "kudu.consensus.ConsensusService.GetLastOpId RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.GetLastOpId() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_GetConsensusState",
+            "label": "kudu.consensus.ConsensusService.GetConsensusState RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.GetConsensusState() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_consensus_ConsensusService_StartTabletCopy",
+            "label": "kudu.consensus.ConsensusService.StartTabletCopy RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.consensus.ConsensusService.StartTabletCopy() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerAdminService_CreateTablet",
+            "label": "kudu.tserver.TabletServerAdminService.CreateTablet RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerAdminService.CreateTablet() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerAdminService_DeleteTablet",
+            "label": "kudu.tserver.TabletServerAdminService.DeleteTablet RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerAdminService.DeleteTablet() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_tserver_TabletServerAdminService_AlterSchema",
+            "label": "kudu.tserver.TabletServerAdminService.AlterSchema RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.tserver.TabletServerAdminService.AlterSchema() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_Ping",
+            "label": "kudu.master.MasterService.Ping RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.Ping() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_TSHeartbeat",
+            "label": "kudu.master.MasterService.TSHeartbeat RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.TSHeartbeat() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_GetTabletLocations",
+            "label": "kudu.master.MasterService.GetTabletLocations RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.GetTabletLocations() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_CreateTable",
+            "label": "kudu.master.MasterService.CreateTable RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.CreateTable() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_IsCreateTableDone",
+            "label": "kudu.master.MasterService.IsCreateTableDone RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.IsCreateTableDone() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_DeleteTable",
+            "label": "kudu.master.MasterService.DeleteTable RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.DeleteTable() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_AlterTable",
+            "label": "kudu.master.MasterService.AlterTable RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.AlterTable() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_IsAlterTableDone",
+            "label": "kudu.master.MasterService.IsAlterTableDone RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.IsAlterTableDone() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_ListTables",
+            "label": "kudu.master.MasterService.ListTables RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.ListTables() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_GetTableLocations",
+            "label": "kudu.master.MasterService.GetTableLocations RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.GetTableLocations() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_GetTableSchema",
+            "label": "kudu.master.MasterService.GetTableSchema RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.GetTableSchema() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_ListTabletServers",
+            "label": "kudu.master.MasterService.ListTabletServers RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.ListTabletServers() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_ListMasters",
+            "label": "kudu.master.MasterService.ListMasters RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.ListMasters() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "handler_latency_kudu_master_MasterService_GetMasterRegistration",
+            "label": "kudu.master.MasterService.GetMasterRegistration RPC Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Microseconds spent handling kudu.master.MasterService.GetMasterRegistration() RPC requests",
+            "entity_type": "server"
+        },
+        {
+            "name": "rpc_connections_accepted",
+            "label": "RPC Connections Accepted",
+            "type": "counter",
+            "unit": "connections",
+            "description": "Number of incoming TCP connections made to the RPC server",
+            "entity_type": "server"
+        },
+        {
+            "name": "rpc_incoming_queue_time",
+            "label": "RPC Queue Time",
+            "type": "histogram",
+            "unit": "microseconds",
+            "description": "Number of microseconds incoming RPC requests spend in the worker queue",
+            "entity_type": "server"
+        },
+        {
+            "name": "rpcs_timed_out_in_queue",
+            "label": "RPC Queue Timeouts",
+            "type": "counter",
+            "unit": "requests",
+            "description": "Number of RPCs whose timeout elapsed while waiting in the service queue, and thus were not processed.",
+            "entity_type": "server"
+        },
+        {
+            "name": "rpcs_queue_overflow",
+            "label": "RPC Queue Overflows",
+            "type": "counter",
+            "unit": "requests",
+            "description": "Number of RPCs dropped because the service queue was full.",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_inserts",
+            "label": "Block Cache Inserts",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of blocks inserted in the cache",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_lookups",
+            "label": "Block Cache Lookups",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of blocks looked up from the cache",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_evictions",
+            "label": "Block Cache Evictions",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of blocks evicted from the cache",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_misses",
+            "label": "Block Cache Misses",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of lookups that didn't yield a block",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_misses_caching",
+            "label": "Block Cache Misses (Caching)",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of lookups that were expecting a block that didn't yield one.Use this number instead of cache_misses when trying to determine how efficient the cache is",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_hits",
+            "label": "Block Cache Hits",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of lookups that found a block",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_hits_caching",
+            "label": "Block Cache Hits (Caching)",
+            "type": "counter",
+            "unit": "blocks",
+            "description": "Number of lookups that were expecting a block that found one.Use this number instead of cache_hits when trying to determine how efficient the cache is",
+            "entity_type": "server"
+        },
+        {
+            "name": "block_cache_usage",
+            "label": "Block Cache Memory Usage",
+            "type": "gauge",
+            "unit": "bytes",
+            "description": "Memory consumed by the block cache",
+            "entity_type": "server"
+        },
+        {
+            "name": "spinlock_contention_time",
+            "label": "Spinlock Contention Time",
+            "type": "counter",
+            "unit": "microseconds",
+            "description": "Amount of time consumed by contention on internal spinlocks since the server started. If this increases rapidly, it may indicate a performance issue in Kudu internals triggered by a particular workload and warrant investigation.",
+            "entity_type": "server"
+        },
+        {
+            "name": "tcmalloc_contention_time",
+            "label": "TCMalloc Contention Time",
+            "type": "counter",
+            "unit": "microseconds",
+            "description": "Amount of time consumed by contention on tcmalloc's locks since the server started. If this increases rapidly, it may indicate a performance issue in Kudu internals triggered by a particular workload and warrant investigation.",
+            "entity_type": "server"
+        },
+        {
+            "name": "threads_started",
+            "label": "Threads Started",
+            "type": "counter",
+            "unit": "threads",
+            "description": "Total number of threads started on this server",
+            "entity_type": "server"
+        },
+        {
+            "name": "threads_running",
+            "label": "Threads Running",
+            "type": "gauge",
+            "unit": "threads",
+            "description": "Current number of running threads",
+            "entity_type": "server"
+        },
+        {
+            "name": "cpu_utime",
+            "label": "User CPU Time",
+            "type": "counter",
+            "unit": "milliseconds",
+            "description": "Total user CPU time of the process",
+            "entity_type": "server"
+        },
+        {
+            "name": "cpu_stime",
+            "label": "System CPU Time",
+            "type": "counter",
+            "unit": "milliseconds",
+            "description": "Total system CPU time of the process",
+            "entity_type": "server"
+        },
+        {
+            "name": "voluntary_context_switches",
+            "label": "Voluntary Context Switches",
+            "type": "counter",
+            "unit": "context switches",
+            "description": "Total voluntary context switches",
+            "entity_type": "server"
+        },
+        {
+            "name": "involuntary_context_switches",
+            "label": "Involuntary Context Switches",
+            "type": "counter",
+            "unit": "context switches",
+            "description": "Total involuntary context switches",
+            "entity_type": "server"
+        }
+    ],
+    "entities": [
+        {
+            "name": "tablet"
+        },
+        {
+            "name": "server"
+        }
+    ]
+}