You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2017/01/25 19:48:44 UTC

[2/3] hbase git commit: HBASE-9774 HBase native metrics and metric collection for coprocessors

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java
new file mode 100644
index 0000000..fccaa38
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Meter.java
@@ -0,0 +1,90 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metric which measure the rate at which some operation is invoked.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Meter extends Metric {
+
+  /**
+   * Records one occurrence.
+   */
+  void mark();
+
+  /**
+   * Records {@code events} occurrences.
+   *
+   * @param events Number of occurrences to record.
+   */
+  void mark(long events);
+
+  /**
+   * Returns the number of events.
+   * @return the number of events.
+   */
+  long getCount();
+
+  /**
+   * Returns the mean rate at which events have occurred since the meter was created.
+   * @return the mean rate at which events have occurred since the meter was created
+   */
+  double getMeanRate();
+
+  /**
+   * Returns the one-minute exponentially-weighted moving average rate at which events have
+   * occurred since the meter was created.
+   * <p/>
+   * This rate has the same exponential decay factor as the one-minute load average in the {@code
+   * top} Unix command.
+   *
+   * @return the one-minute exponentially-weighted moving average rate at which events have
+   *         occurred since the meter was created
+   */
+  double getOneMinuteRate();
+
+  /**
+   * Returns the five-minute exponentially-weighted moving average rate at which events have
+   * occurred since the meter was created.
+   * <p/>
+   * This rate has the same exponential decay factor as the five-minute load average in the {@code
+   * top} Unix command.
+   *
+   * @return the five-minute exponentially-weighted moving average rate at which events have
+   *         occurred since the meter was created
+   */
+  double getFiveMinuteRate();
+
+  /**
+   * Returns the fifteen-minute exponentially-weighted moving average rate at which events have
+   * occurred since the meter was created.
+   * <p/>
+   * This rate has the same exponential decay factor as the fifteen-minute load average in the
+   * {@code top} Unix command.
+   *
+   * @return the fifteen-minute exponentially-weighted moving average rate at which events have
+   *         occurred since the meter was created
+   */
+  double getFifteenMinuteRate();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java
new file mode 100644
index 0000000..0a31803
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Metric.java
@@ -0,0 +1,30 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Parent interface for all metrics.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Metric {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java
new file mode 100644
index 0000000..502119d
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistries.java
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.Collection;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * MetricRegistries is collection of MetricRegistry's. MetricsRegistries implementations should do
+ * ref-counting of MetricRegistry's via create() and remove() methods.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public abstract class MetricRegistries {
+
+  private static final class LazyHolder {
+    private static final MetricRegistries GLOBAL = MetricRegistriesLoader.load();
+  }
+
+  /**
+   * Return the global singleton instance for the MetricRegistries.
+   * @return MetricRegistries implementation.
+   */
+  public static MetricRegistries global() {
+    return LazyHolder.GLOBAL;
+  }
+
+  /**
+   * Removes all the MetricRegisties.
+   */
+  public abstract void clear();
+
+  /**
+   * Create or return MetricRegistry with the given info. MetricRegistry will only be created
+   * if current reference count is 0. Otherwise ref counted is incremented, and an existing instance
+   * will be returned.
+   * @param info the info object for the MetricRegistrytry.
+   * @return created or existing MetricRegistry.
+   */
+  public abstract MetricRegistry create(MetricRegistryInfo info);
+
+  /**
+   * Decrements the ref count of the MetricRegistry, and removes if ref count == 0.
+   * @param key the info object for the MetricRegistrytry.
+   * @return true if metric registry is removed.
+   */
+  public abstract boolean remove(MetricRegistryInfo key);
+
+  /**
+   * Returns the MetricRegistry if found.
+   * @param info the info for the registry.
+   * @return a MetricRegistry optional.
+   */
+  public abstract Optional<MetricRegistry> get(MetricRegistryInfo info);
+
+  /**
+   * Returns MetricRegistryInfo's for the MetricRegistry's created.
+   * @return MetricRegistryInfo's for the MetricRegistry's created.
+   */
+  public abstract Set<MetricRegistryInfo> getMetricRegistryInfos();
+
+  /**
+   * Returns MetricRegistry's created.
+   * @return MetricRegistry's created.
+   */
+  public abstract Collection<MetricRegistry> getMetricRegistries();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
new file mode 100644
index 0000000..4fef10c
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistriesLoader.java
@@ -0,0 +1,96 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ServiceLoader;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+
+@InterfaceAudience.Private
+public class MetricRegistriesLoader {
+  private static final Log LOG = LogFactory.getLog(MetricRegistries.class);
+
+  private static final String defaultClass
+      = "org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl";
+
+  /**
+   * Creates a {@link MetricRegistries} instance using the corresponding {@link MetricRegistries}
+   * available to {@link ServiceLoader} on the classpath. If no instance is found, then default
+   * implementation will be loaded.
+   * @return A {@link MetricRegistries} implementation.
+   */
+  public static MetricRegistries load() {
+    List<MetricRegistries> availableImplementations = getDefinedImplemantations();
+    return load(availableImplementations);
+  }
+
+  /**
+   * Creates a {@link MetricRegistries} instance using the corresponding {@link MetricRegistries}
+   * available to {@link ServiceLoader} on the classpath. If no instance is found, then default
+   * implementation will be loaded.
+   * @return A {@link MetricRegistries} implementation.
+   */
+  @VisibleForTesting
+  static MetricRegistries load(List<MetricRegistries> availableImplementations) {
+
+    if (availableImplementations.size() == 1) {
+      // One and only one instance -- what we want/expect
+      MetricRegistries impl = availableImplementations.get(0);
+      LOG.info("Loaded MetricRegistries " + impl.getClass());
+      return impl;
+    } else if (availableImplementations.isEmpty()) {
+      try {
+        return ReflectionUtils.newInstance((Class<MetricRegistries>)Class.forName(defaultClass));
+      } catch (ClassNotFoundException e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      // Tell the user they're doing something wrong, and choose the first impl.
+      StringBuilder sb = new StringBuilder();
+      for (MetricRegistries factory : availableImplementations) {
+        if (sb.length() > 0) {
+          sb.append(", ");
+        }
+        sb.append(factory.getClass());
+      }
+      LOG.warn("Found multiple MetricRegistries implementations: " + sb
+          + ". Using first found implementation: " + availableImplementations.get(0));
+      return availableImplementations.get(0);
+    }
+  }
+
+  private static List<MetricRegistries> getDefinedImplemantations() {
+    ServiceLoader<MetricRegistries> loader = ServiceLoader.load(MetricRegistries.class);
+    List<MetricRegistries> availableFactories = new ArrayList<>();
+    for (MetricRegistries impl : loader) {
+      availableFactories.add(impl);
+    }
+    return availableFactories;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java
new file mode 100644
index 0000000..3138bca
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistry.java
@@ -0,0 +1,111 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.Optional;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * General purpose factory for creating various metrics.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface MetricRegistry extends MetricSet {
+
+  /**
+   * Get or construct a {@link Timer} used to measure durations and report rates.
+   *
+   * @param name the name of the timer.
+   * @return An instance of {@link Timer}.
+   */
+  Timer timer(String name);
+
+  /**
+   * Get or construct a {@link Histogram} used to measure a distribution of values.
+   *
+   * @param name The name of the Histogram.
+   * @return An instance of {@link Histogram}.
+   */
+  Histogram histogram(String name);
+
+  /**
+   * Get or construct a {@link Meter} used to measure durations and report distributions (a
+   * combination of a {@link Timer} and a {@link Histogram}.
+   *
+   * @param name The name of the Meter.
+   * @return An instance of {@link Meter}.
+   */
+  Meter meter(String name);
+
+  /**
+   * Get or construct a {@link Counter} used to track a mutable number.
+   *
+   * @param name The name of the Counter
+   * @return An instance of {@link Counter}.
+   */
+  Counter counter(String name);
+
+  /**
+   * Register a {@link Gauge}. The Gauge will be invoked at a period defined by the implementation
+   * of {@link MetricRegistry}.
+   * @param name The name of the Gauge.
+   * @param gauge A callback to compute the current value.
+   * @return the registered gauge, or the existing gauge
+   */
+  <T> Gauge<T> register(String name, Gauge<T> gauge);
+
+  /**
+   * Registers the {@link Metric} with the given name if there does not exist one with the same
+   * name. Returns the newly registered or existing Metric.
+   * @param name The name of the Metric.
+   * @param metric the metric to register
+   * @return the registered metric, or the existing metrid
+   */
+  Metric register(String name, Metric metric);
+
+  /**
+   * Registers the {@link Metric}s in the given MetricSet.
+   * @param metricSet set of metrics to register.
+   */
+  void registerAll(MetricSet metricSet);
+
+  /**
+   * Returns previously registered metric with the name if any.
+   * @param name the name of the metric
+   * @return previously registered metric
+   */
+  Optional<Metric> get(String name);
+
+  /**
+   * Removes the metric with the given name.
+   *
+   * @param name the name of the metric
+   * @return true if the metric is removed.
+   */
+  boolean remove(String name);
+
+  /**
+   * Return the MetricRegistryInfo object for this registry.
+   * @return MetricRegistryInfo describing the registry.
+   */
+  @InterfaceAudience.Private
+  MetricRegistryInfo getMetricRegistryInfo();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java
new file mode 100644
index 0000000..b161d20
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryFactory.java
@@ -0,0 +1,36 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A Factory for creating MetricRegistries. This is the main plugin point for metrics implementation
+ */
+@InterfaceAudience.Private
+public interface MetricRegistryFactory {
+  /**
+   * Create a MetricRegistry from the given MetricRegistryInfo
+   * @param info the descriptor for MetricRegistry
+   * @return a MetricRegistry implementation
+   */
+  MetricRegistry create(MetricRegistryInfo info);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java
new file mode 100644
index 0000000..58fcba7
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricRegistryInfo.java
@@ -0,0 +1,112 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.metrics;
+
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * HBase Metrics are grouped in different MetricRegistry'ies. All metrics that correspond to a
+ * subcomponent (like RPC, GC, WAL) are managed in a single MetricRegistry.
+ * This class holds the name and description and JMX related context names for such group of
+ * metrics.
+ */
+@InterfaceAudience.Private
+public class MetricRegistryInfo {
+
+  protected final String metricsName;
+  protected final String metricsDescription;
+  protected final String metricsContext;
+  protected final String metricsJmxContext;
+  protected final boolean existingSource;
+
+  public MetricRegistryInfo(
+      String metricsName,
+      String metricsDescription,
+      String metricsJmxContext,
+      String metricsContext,
+      boolean existingSource) {
+    this.metricsName = metricsName;
+    this.metricsDescription = metricsDescription;
+    this.metricsContext = metricsContext;
+    this.metricsJmxContext = metricsJmxContext;
+    this.existingSource = existingSource;
+  }
+
+  /**
+   * Get the metrics context.  For hadoop metrics2 system this is usually an all lowercased string.
+   * eg. regionserver, master, thriftserver
+   *
+   * @return The string context used to register this source to hadoop's metrics2 system.
+   */
+  public String getMetricsContext() {
+    return metricsContext;
+  }
+
+  /**
+   * Get the description of what this source exposes.
+   */
+  public String getMetricsDescription() {
+    return metricsDescription;
+  }
+
+  /**
+   * Get the name of the context in JMX that this source will be exposed through.
+   * This is in ObjectName format. With the default context being Hadoop -&gt; HBase
+   */
+  public String getMetricsJmxContext() {
+    return metricsJmxContext;
+  }
+
+  /**
+   * Get the name of the metrics that are being exported by this source.
+   * Eg. IPC, GC, WAL
+   */
+  public String getMetricsName() {
+    return metricsName;
+  }
+
+  /**
+   * Returns whether or not this MetricRegistry is for an existing BaseSource
+   * @return true if this MetricRegistry is for an existing BaseSource.
+   */
+  public boolean isExistingSource() {
+    return existingSource;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof MetricRegistryInfo) {
+      return this.hashCode() == obj.hashCode();
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder()
+        .append(metricsName)
+        .append(metricsDescription)
+        .append(metricsContext)
+        .append(metricsJmxContext)
+        .hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
new file mode 100644
index 0000000..9cf2378
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/MetricSet.java
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A set of named metrics.
+ *
+ * @see MetricRegistry#registerAll(MetricSet)
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface MetricSet extends Metric {
+    /**
+     * A map of metric names to metrics.
+     *
+     * @return the metrics
+     */
+    Map<String, Metric> getMetrics();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java
new file mode 100644
index 0000000..8278d0c
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/PackageMarker.java
@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * This is a dummy annotation that forces javac to produce output for
+ * otherwise empty package-info.java.
+ *
+ * <p>The result is maven-compiler-plugin can properly identify the scope of
+ * changed files
+ *
+ * <p>See more details in
+ * <a href="https://jira.codehaus.org/browse/MCOMPILER-205">
+ *   maven-compiler-plugin: incremental compilation broken</a>
+ */
+@Retention(RetentionPolicy.SOURCE)
+public @interface PackageMarker {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java
new file mode 100644
index 0000000..56ee8ae
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Snapshot.java
@@ -0,0 +1,136 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A statictical sample of histogram values.
+ */
+@InterfaceAudience.Private
+public interface Snapshot {
+
+  /**
+   * Return the values with the given quantiles.
+   * @param quantiles the requested quantiles.
+   * @return the value for the quantiles.
+   */
+  long[] getQuantiles(double[] quantiles);
+
+  /**
+   * Return the values with the default quantiles.
+   * @return the value for default the quantiles.
+   */
+  long[] getQuantiles();
+
+  /**
+   * Returns the number of values in the snapshot.
+   *
+   * @return the number of values
+   */
+  long getCount();
+
+  /**
+   * Returns the total count below the given value
+   * @param val the value
+   * @return the total count below the given value
+   */
+  long getCountAtOrBelow(long val);
+
+  /**
+   * Returns the value at the 25th percentile in the distribution.
+   *
+   * @return the value at the 25th percentile
+   */
+  long get25thPercentile();
+
+  /**
+   * Returns the value at the 75th percentile in the distribution.
+   *
+   * @return the value at the 75th percentile
+   */
+  long get75thPercentile();
+
+  /**
+   * Returns the value at the 90th percentile in the distribution.
+   *
+   * @return the value at the 90th percentile
+   */
+  long get90thPercentile();
+
+  /**
+   * Returns the value at the 95th percentile in the distribution.
+   *
+   * @return the value at the 95th percentile
+   */
+  long get95thPercentile();
+
+  /**
+   * Returns the value at the 98th percentile in the distribution.
+   *
+   * @return the value at the 98th percentile
+   */
+  long get98thPercentile();
+
+  /**
+   * Returns the value at the 99th percentile in the distribution.
+   *
+   * @return the value at the 99th percentile
+   */
+  long get99thPercentile();
+
+  /**
+   * Returns the value at the 99.9th percentile in the distribution.
+   *
+   * @return the value at the 99.9th percentile
+   */
+  long get999thPercentile();
+
+  /**
+   * Returns the median value in the distribution.
+   *
+   * @return the median value
+   */
+  long getMedian();
+
+  /**
+   * Returns the highest value in the snapshot.
+   *
+   * @return the highest value
+   */
+  long getMax();
+
+  /**
+   * Returns the arithmetic mean of the values in the snapshot.
+   *
+   * @return the arithmetic mean
+   */
+  long getMean();
+
+  /**
+   * Returns the lowest value in the snapshot.
+   *
+   * @return the lowest value
+   */
+  long getMin();
+
+  // TODO: Dropwizard histograms also track stddev
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java
new file mode 100644
index 0000000..63fc649
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Timer.java
@@ -0,0 +1,68 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metric which encompasses a {@link Histogram} and {@link Meter}.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Timer extends Metric {
+  /**
+   * Update the timer with the given duration in given time unit.
+   * @param duration the duration of the event
+   * @param unit the time unit for the duration
+   */
+  void update(long duration, TimeUnit unit);
+
+  /**
+   * Update the timer with the given duration in milliseconds
+   * @param durationMillis the duration of the event in ms
+   */
+  default void updateMillis(long durationMillis) {
+    update(durationMillis, TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Update the timer with the given duration in microseconds
+   * @param durationMicros the duration of the event in microseconds
+   */
+  default void updateMicros(long durationMicros) {
+    update(durationMicros, TimeUnit.MICROSECONDS);
+  }
+
+  /**
+   * Update the timer with the given duration in nanoseconds
+   * @param durationNanos the duration of the event in ns
+   */
+  default void updateNanos(long durationNanos) {
+    update(durationNanos, TimeUnit.NANOSECONDS);
+  }
+
+  @InterfaceAudience.Private
+  Histogram getHistogram();
+
+  @InterfaceAudience.Private
+  Meter getMeter();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java
new file mode 100644
index 0000000..e79451f
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+/**
+ * Metrics API for HBase.
+ */
+@PackageMarker
+package org.apache.hadoop.hbase.metrics;
+
+// End package-info.java

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java b/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java
new file mode 100644
index 0000000..8746146
--- /dev/null
+++ b/hbase-metrics-api/src/test/java/org/apache/hadoop/hbase/metrics/TestMetricRegistriesLoader.java
@@ -0,0 +1,56 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Test class for {@link MetricRegistriesLoader}.
+ */
+@Category(SmallTests.class)
+public class TestMetricRegistriesLoader {
+
+  @Test
+  public void testLoadSinleInstance() {
+    MetricRegistries loader = mock(MetricRegistries.class);
+    MetricRegistries instance = MetricRegistriesLoader.load(Lists.newArrayList(loader));
+    assertEquals(loader, instance);
+  }
+
+  @Test
+  public void testLoadMultipleInstances() {
+    MetricRegistries loader1 = mock(MetricRegistries.class);
+    MetricRegistries loader2 = mock(MetricRegistries.class);
+    MetricRegistries loader3 = mock(MetricRegistries.class);
+    MetricRegistries instance = MetricRegistriesLoader.load(Lists.newArrayList(loader1, loader2,
+        loader3));
+
+    // the load() returns the first instance
+    assertEquals(loader1, instance);
+    assertNotEquals(loader2, instance);
+    assertNotEquals(loader3, instance);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/README.txt
----------------------------------------------------------------------
diff --git a/hbase-metrics/README.txt b/hbase-metrics/README.txt
new file mode 100644
index 0000000..d80064c
--- /dev/null
+++ b/hbase-metrics/README.txt
@@ -0,0 +1 @@
+See the documentation at hbase-metrics-api/README.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics/pom.xml b/hbase-metrics/pom.xml
new file mode 100644
index 0000000..fbd2bd5
--- /dev/null
+++ b/hbase-metrics/pom.xml
@@ -0,0 +1,136 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+  /**
+   * 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.
+   */
+  -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>hbase-metrics</artifactId>
+  <name>Apache HBase - Metrics Implementation</name>
+  <description>HBase Metrics Implementation</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven.assembly.version}</version>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <!-- Always skip the second part executions, since we only run simple unit tests in this module -->
+        <executions>
+          <execution>
+            <id>secondPartTestsExecution</id>
+            <phase>test</phase>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-metrics-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-metrics-api</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <!-- Skip the tests in this module -->
+    <profile>
+      <id>skip-metrics-tests</id>
+      <activation>
+        <property>
+          <name>skip-metrics-tests</name>
+        </property>
+      </activation>
+      <properties>
+        <surefire.skipFirstPart>true</surefire.skipFirstPart>
+      </properties>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
new file mode 100644
index 0000000..147a903
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.concurrent.atomic.LongAdder;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Counter;
+
+/**
+ * Custom implementation of {@link org.apache.hadoop.hbase.metrics.Counter} using LongAdder.
+ */
+@InterfaceAudience.Private
+public class CounterImpl implements Counter {
+
+  private final LongAdder counter;
+
+  public CounterImpl() {
+    this.counter = new LongAdder();
+  }
+
+  @Override
+  public void increment() {
+    this.counter.increment();
+  }
+
+  @Override
+  public void increment(long n) {
+    this.counter.add(n);
+  }
+
+  @Override
+  public void decrement() {
+    this.counter.decrement();
+  }
+
+  @Override
+  public void decrement(long n) {
+    this.counter.add(-n);
+  }
+
+  @Override
+  public long getCount() {
+    return this.counter.sum();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
new file mode 100644
index 0000000..fc92483
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
@@ -0,0 +1,74 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.Objects;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import com.codahale.metrics.Meter;
+
+/**
+ * Dropwizard metrics implementation of {@link org.apache.hadoop.hbase.metrics.Meter}.
+ */
+@InterfaceAudience.Private
+public class DropwizardMeter implements org.apache.hadoop.hbase.metrics.Meter {
+
+  private final Meter meter;
+
+  public DropwizardMeter() {
+    this.meter = new Meter();
+  }
+
+  public DropwizardMeter(Meter meter) {
+    this.meter = Objects.requireNonNull(meter);
+  }
+
+  @Override public void mark() {
+    this.meter.mark();
+  }
+
+  @Override public void mark(long count) {
+    this.meter.mark(count);
+  }
+
+  @Override
+  public long getCount() {
+    return meter.getCount();
+  }
+
+  @Override
+  public double getMeanRate() {
+    return meter.getMeanRate();
+  }
+
+  @Override
+  public double getOneMinuteRate() {
+    return meter.getOneMinuteRate();
+  }
+
+  @Override
+  public double getFiveMinuteRate() {
+    return meter.getFiveMinuteRate();
+  }
+
+  @Override
+  public double getFifteenMinuteRate() {
+    return meter.getFifteenMinuteRate();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
new file mode 100644
index 0000000..aa28193
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
@@ -0,0 +1,397 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.util.AtomicUtils;
+
+/**
+ * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the
+ * quantiles.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class FastLongHistogram {
+
+  /**
+   * Default number of bins.
+   */
+  public static final int DEFAULT_NBINS = 255;
+
+  public static final double[] DEFAULT_QUANTILES =
+      new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
+
+  /**
+   * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
+   */
+  private static class Bins {
+    private final LongAdder[] counts;
+    // inclusive
+    private final long binsMin;
+    // exclusive
+    private final long binsMax;
+    private final long bins10XMax;
+    private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
+    private final AtomicLong max = new AtomicLong(0L);
+
+    private final LongAdder count = new LongAdder();
+    private final LongAdder total = new LongAdder();
+
+    // set to true when any of data has been inserted to the Bins. It is set after the counts are
+    // updated.
+    private volatile boolean hasData = false;
+
+    /**
+     * The constructor for creating a Bins without any prior data.
+     */
+    public Bins(int numBins) {
+      counts = createCounters(numBins);
+      this.binsMin = 1L;
+
+      // These two numbers are total guesses
+      // and should be treated as highly suspect.
+      this.binsMax = 1000;
+      this.bins10XMax = binsMax * 10;
+    }
+
+    /**
+     * The constructor for creating a Bins with last Bins.
+     */
+    public Bins(Bins last, int numBins, double minQ, double maxQ) {
+      long[] values = last.getQuantiles(new double[] { minQ, maxQ });
+      long wd = values[1] - values[0] + 1;
+      // expand minQ and maxQ in two ends back assuming uniform distribution
+      this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
+      long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
+      // make sure each of bins is at least of width 1
+      this.binsMax = Math.max(binsMax, this.binsMin + numBins);
+      this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
+
+      this.counts = createCounters(numBins);
+    }
+
+    private LongAdder[] createCounters(int numBins) {
+      return Stream.generate(LongAdder::new).limit(numBins + 3).toArray(LongAdder[]::new);
+    }
+
+    private int getIndex(long value) {
+      if (value < this.binsMin) {
+        return 0;
+      } else if (value > this.bins10XMax) {
+        return this.counts.length - 1;
+      } else if (value >= this.binsMax) {
+        return this.counts.length - 2;
+      }
+      // compute the position
+      return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
+          (this.binsMax - this.binsMin));
+
+    }
+
+    /**
+     * Adds a value to the histogram.
+     */
+    public void add(long value, long count) {
+      if (value < 0) {
+        // The whole computation is completely thrown off if there are negative numbers
+        //
+        // Normally we would throw an IllegalArgumentException however this is the metrics
+        // system and it should be completely safe at all times.
+        // So silently throw it away.
+        return;
+      }
+      AtomicUtils.updateMin(min, value);
+      AtomicUtils.updateMax(max, value);
+
+      this.count.add(count);
+      this.total.add(value * count);
+
+      int pos = getIndex(value);
+      this.counts[pos].add(count);
+
+      // hasData needs to be updated as last
+      this.hasData = true;
+    }
+
+    /**
+     * Computes the quantiles give the ratios.
+     */
+    public long[] getQuantiles(double[] quantiles) {
+      if (!hasData) {
+        // No data yet.
+        return new long[quantiles.length];
+      }
+
+      // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
+      // This is not synchronized, but since the counter are accumulating, the result is a good
+      // estimation of a snapshot.
+      long[] counts = new long[this.counts.length];
+      long total = 0L;
+      for (int i = 0; i < this.counts.length; i++) {
+        counts[i] = this.counts[i].sum();
+        total += counts[i];
+      }
+
+      int rIndex = 0;
+      double qCount = total * quantiles[0];
+      long cum = 0L;
+
+      long[] res = new long[quantiles.length];
+      countsLoop: for (int i = 0; i < counts.length; i++) {
+        // mn and mx define a value range
+        long mn, mx;
+        if (i == 0) {
+          mn = this.min.get();
+          mx = this.binsMin;
+        } else if (i == counts.length - 1) {
+          mn = this.bins10XMax;
+          mx = this.max.get();
+        } else if (i == counts.length - 2) {
+          mn = this.binsMax;
+          mx = this.bins10XMax;
+        } else {
+          mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+          mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+        }
+
+        if (mx < this.min.get()) {
+          continue;
+        }
+        if (mn > this.max.get()) {
+          break;
+        }
+        mn = Math.max(mn, this.min.get());
+        mx = Math.min(mx, this.max.get());
+
+        // lastCum/cum are the corresponding counts to mn/mx
+        double lastCum = cum;
+        cum += counts[i];
+
+        // fill the results for qCount is within current range.
+        while (qCount <= cum) {
+          if (cum == lastCum) {
+            res[rIndex] = mn;
+          } else {
+            res[rIndex] = (long) ((qCount - lastCum) * (mx - mn) / (cum - lastCum) + mn);
+          }
+
+          // move to next quantile
+          rIndex++;
+          if (rIndex >= quantiles.length) {
+            break countsLoop;
+          }
+          qCount = total * quantiles[rIndex];
+        }
+      }
+      // In case quantiles contains values >= 100%
+      for (; rIndex < quantiles.length; rIndex++) {
+        res[rIndex] = this.max.get();
+      }
+
+      return res;
+    }
+
+    long getNumAtOrBelow(long val) {
+      return Arrays.stream(counts).mapToLong(c -> c.sum()).limit(getIndex(val) + 1).sum();
+    }
+
+    public long getMin() {
+      long min = this.min.get();
+      return min == Long.MAX_VALUE ? 0 : min; // in case it is not initialized
+    }
+
+    public long getMean() {
+      long count = this.count.sum();
+      long total = this.total.sum();
+      if (count == 0) {
+        return 0;
+      }
+      return total / count;
+    }
+  }
+
+  // The bins counting values. It is replaced with a new one in calling of reset().
+  private volatile Bins bins;
+
+  /**
+   * Constructor.
+   */
+  public FastLongHistogram() {
+    this(DEFAULT_NBINS);
+  }
+
+  /**
+   * Constructor.
+   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
+   *          results but with lower efficiency, and vice versus.
+   */
+  public FastLongHistogram(int numOfBins) {
+    this.bins = new Bins(numOfBins);
+  }
+
+  /**
+   * Constructor setting the bins assuming a uniform distribution within a range.
+   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
+   *          results but with lower efficiency, and vice versus.
+   * @param min lower bound of the region, inclusive.
+   * @param max higher bound of the region, inclusive.
+   */
+  public FastLongHistogram(int numOfBins, long min, long max) {
+    this(numOfBins);
+    Bins bins = new Bins(numOfBins);
+    bins.add(min, 1);
+    bins.add(max, 1);
+    this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
+  }
+
+  private FastLongHistogram(Bins bins) {
+    this.bins = bins;
+  }
+
+  /**
+   * Adds a value to the histogram.
+   */
+  public void add(long value, long count) {
+    this.bins.add(value, count);
+  }
+
+  /**
+   * Computes the quantiles give the ratios.
+   */
+  public long[] getQuantiles(double[] quantiles) {
+    return this.bins.getQuantiles(quantiles);
+  }
+
+  public long[] getQuantiles() {
+    return this.bins.getQuantiles(DEFAULT_QUANTILES);
+  }
+
+  public long getMin() {
+    return this.bins.getMin();
+  }
+
+  public long getMax() {
+    return this.bins.max.get();
+  }
+
+  public long getCount() {
+    return this.bins.count.sum();
+  }
+
+  public long getMean() {
+    return this.bins.getMean();
+  }
+
+  public long getNumAtOrBelow(long value) {
+    return this.bins.getNumAtOrBelow(value);
+  }
+
+  /**
+   * Resets the histogram for new counting.
+   */
+  public Snapshot snapshotAndReset() {
+    final Bins oldBins = this.bins;
+    this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
+    final long[] percentiles = oldBins.getQuantiles(DEFAULT_QUANTILES);
+    final long count = oldBins.count.sum();
+
+    return new Snapshot() {
+      @Override
+      public long[] getQuantiles(double[] quantiles) {
+        return oldBins.getQuantiles(quantiles);
+      }
+
+      @Override
+      public long[] getQuantiles() {
+        return percentiles;
+      }
+
+      @Override
+      public long getCount() {
+        return count;
+      }
+
+      @Override
+      public long getCountAtOrBelow(long val) {
+        return oldBins.getNumAtOrBelow(val);
+      }
+
+      @Override
+      public long get25thPercentile() {
+        return percentiles[0];
+      }
+
+      @Override
+      public long get75thPercentile() {
+        return percentiles[2];
+      }
+
+      @Override
+      public long get90thPercentile() {
+        return percentiles[3];
+      }
+
+      @Override
+      public long get95thPercentile() {
+        return percentiles[4];
+      }
+
+      @Override
+      public long get98thPercentile() {
+        return percentiles[5];
+      }
+
+      @Override
+      public long get99thPercentile() {
+        return percentiles[6];
+      }
+
+      @Override
+      public long get999thPercentile() {
+        return percentiles[7];
+      }
+
+      @Override
+      public long getMedian() {
+        return percentiles[1];
+      }
+
+      @Override
+      public long getMax() {
+        return oldBins.max.get();
+      }
+
+      @Override
+      public long getMean() {
+        return oldBins.getMean();
+      }
+
+      @Override
+      public long getMin() {
+        return oldBins.getMin();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
new file mode 100644
index 0000000..b52caf8
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
@@ -0,0 +1,81 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+
+/**
+ * Custom histogram implementation based on FastLongHistogram. Dropwizard-based histograms are
+ * slow compared to this implementation, so we are using our implementation here.
+ * See HBASE-15222.
+ */
+@InterfaceAudience.Private
+public class HistogramImpl implements Histogram {
+  // Double buffer the two FastLongHistograms.
+  // As they are reset they learn how the buckets should be spaced
+  // So keep two around and use them
+  protected final FastLongHistogram histogram;
+  private final CounterImpl counter;
+
+  public HistogramImpl() {
+    this(Integer.MAX_VALUE << 2);
+  }
+
+  public HistogramImpl(long maxExpected) {
+    this(FastLongHistogram.DEFAULT_NBINS, 1, maxExpected);
+  }
+
+  public HistogramImpl(int numBins, long min, long maxExpected) {
+    this.counter = new CounterImpl();
+    this.histogram = new FastLongHistogram(numBins, min, maxExpected);
+  }
+
+  protected HistogramImpl(CounterImpl counter, FastLongHistogram histogram) {
+    this.counter = counter;
+    this.histogram = histogram;
+  }
+
+  @Override
+  public void update(int value) {
+    counter.increment();
+    histogram.add(value, 1);
+  }
+
+  @Override
+  public void update(long value) {
+    counter.increment();
+    histogram.add(value, 1);
+  }
+
+  public long getCount() {
+    return counter.getCount();
+  }
+
+  public long getMax() {
+    return this.histogram.getMax();
+  }
+
+  public Snapshot snapshot() {
+    return histogram.snapshotAndReset();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
new file mode 100644
index 0000000..543ed1e
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
@@ -0,0 +1,75 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+
+/**
+ * Implementation of MetricRegistries that does ref-counting.
+ */
+@InterfaceAudience.Private
+public class MetricRegistriesImpl extends MetricRegistries {
+  private final MetricRegistryFactory factory;
+  private final RefCountingMap<MetricRegistryInfo, MetricRegistry> registries;
+
+  public MetricRegistriesImpl() {
+    this(new MetricRegistryFactoryImpl());
+  }
+
+  public MetricRegistriesImpl(MetricRegistryFactory factory) {
+    this.factory = factory;
+    this.registries = new RefCountingMap<>();
+  }
+
+  @Override
+  public MetricRegistry create(MetricRegistryInfo info) {
+    return registries.put(info, () -> factory.create(info));
+  }
+
+  public boolean remove(MetricRegistryInfo key) {
+    return registries.remove(key) == null;
+  }
+
+  public Optional<MetricRegistry> get(MetricRegistryInfo info) {
+    return Optional.ofNullable(registries.get(info));
+  }
+
+  public Collection<MetricRegistry> getMetricRegistries() {
+    return Collections.unmodifiableCollection(registries.values());
+  }
+
+  public void clear() {
+    registries.clear();
+  }
+
+  public Set<MetricRegistryInfo> getMetricRegistryInfos() {
+    return Collections.unmodifiableSet(registries.keySet());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
new file mode 100644
index 0000000..b47153e
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
@@ -0,0 +1,34 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+
+@InterfaceAudience.Private
+public class MetricRegistryFactoryImpl implements MetricRegistryFactory {
+  @Override
+  public MetricRegistry create(MetricRegistryInfo info) {
+    return new MetricRegistryImpl(info);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
new file mode 100644
index 0000000..e7ba755
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
@@ -0,0 +1,122 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.MetricSet;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+
+/**
+ * Custom implementation of {@link MetricRegistry}.
+ */
+@InterfaceAudience.Private
+public class MetricRegistryImpl implements MetricRegistry {
+
+  private final MetricRegistryInfo info;
+
+  private final ConcurrentMap<String, Metric> metrics;
+
+  public MetricRegistryImpl(MetricRegistryInfo info) {
+    this.info = info;
+    this.metrics = new ConcurrentHashMap<>();
+  }
+
+  @Override
+  public Timer timer(String name) {
+    return (Timer) CollectionUtils.computeIfAbsent(metrics, name, this::createTimer);
+  }
+
+  protected Timer createTimer() {
+    return new TimerImpl();
+  }
+
+  @Override
+  public Histogram histogram(String name) {
+    return (Histogram) CollectionUtils.computeIfAbsent(metrics, name, this::createHistogram);
+  }
+
+  protected Histogram createHistogram() {
+    return new HistogramImpl();
+  }
+
+  @Override
+  public Meter meter(String name) {
+    return (Meter) CollectionUtils.computeIfAbsent(metrics, name, this::createMeter);
+  }
+
+  protected Meter createMeter() {
+    return new DropwizardMeter();
+  }
+
+  @Override
+  public Counter counter(String name) {
+    return (Counter) CollectionUtils.computeIfAbsent(metrics, name, this::createCounter);
+  }
+
+  protected Counter createCounter() {
+    return new CounterImpl();
+  }
+
+  @Override
+  public Optional<Metric> get(String name) {
+    return Optional.ofNullable(metrics.get(name));
+  }
+
+  @Override
+  public Metric register(String name, Metric metric) {
+    return CollectionUtils.computeIfAbsent(metrics, name, () -> metric);
+  }
+
+  @Override
+  public <T> Gauge<T> register(String name, Gauge<T> gauge) {
+    return (Gauge) register(name, (Metric)gauge);
+  }
+
+  @Override
+  public void registerAll(MetricSet metricSet) {
+    metricSet.getMetrics().forEach(this::register);
+  }
+
+  @Override
+  public Map<String, Metric> getMetrics() {
+    return metrics;
+  }
+
+  @Override
+  public boolean remove(String name) {
+    return metrics.remove(name) != null;
+  }
+
+  @Override
+  public MetricRegistryInfo getMetricRegistryInfo() {
+    return info;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
new file mode 100644
index 0000000..b185175
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
@@ -0,0 +1,91 @@
+/**
+ *
+ * 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.
+ */
+
+
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * A map of K to V, but does ref counting for added and removed values. The values are
+ * not added directly, but instead requested from the given Supplier if ref count == 0. Each put()
+ * call will increment the ref count, and each remove() will decrement it. The values are removed
+ * from the map iff ref count == 0.
+ */
+@InterfaceAudience.Private
+class RefCountingMap<K, V> {
+
+  private ConcurrentHashMap<K, Payload<V>> map = new ConcurrentHashMap<>();
+  private static class Payload<V> {
+    V v;
+    int refCount;
+    Payload(V v) {
+      this.v = v;
+      this.refCount = 1; // create with ref count = 1
+    }
+  }
+
+  V put(K k, Supplier<V> supplier) {
+    return ((Payload<V>)map.compute(k, (k1, oldValue) -> {
+      if (oldValue != null) {
+        oldValue.refCount++;
+        return oldValue;
+      } else {
+        return new Payload(supplier.get());
+      }
+    })).v;
+  }
+
+  V get(K k) {
+    Payload<V> p = map.get(k);
+    return p == null ? null : p.v;
+  }
+
+  /**
+   * Decrements the ref count of k, and removes from map if ref count == 0.
+   * @param k the key to remove
+   * @return the value associated with the specified key or null if key is removed from map.
+   */
+  V remove(K k) {
+    Payload<V> p = map.computeIfPresent(k, (k1, v) -> --v.refCount <= 0 ? null : v );
+    return p == null ? null : p.v;
+  }
+
+  void clear() {
+    map.clear();
+  }
+
+  Set<K> keySet() {
+    return map.keySet();
+  }
+
+  Collection<V> values() {
+    return map.values().stream().map(v -> v.v).collect(Collectors.toList());
+  }
+
+  int size() {
+    return map.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
new file mode 100644
index 0000000..df2cc84
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
@@ -0,0 +1,58 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+/**
+ * Custom implementation of {@link Timer}.
+ */
+@InterfaceAudience.Private
+public class TimerImpl implements Timer {
+  private final HistogramImpl histogram;
+  private final DropwizardMeter meter;
+
+  // track time events in micros
+  private static final TimeUnit DEFAULT_UNIT = TimeUnit.MICROSECONDS;
+
+  public TimerImpl() {
+    this.histogram = new HistogramImpl();
+    this.meter = new DropwizardMeter();
+  }
+
+  @Override
+  public void update(long duration, TimeUnit unit) {
+    if (duration >= 0) {
+      histogram.update(DEFAULT_UNIT.convert(duration, unit));
+      meter.mark();
+    }
+  }
+
+  @Override
+  public HistogramImpl getHistogram() {
+    return histogram;
+  }
+
+  @Override
+  public DropwizardMeter getMeter() {
+    return meter;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java
new file mode 100644
index 0000000..9c2e952
--- /dev/null
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+/**
+ * Implementation of the HBase Metrics framework.
+ */
+@PackageMarker
+package org.apache.hadoop.hbase.metrics.impl;
+
+import org.apache.hadoop.hbase.metrics.PackageMarker;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries b/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries
new file mode 100644
index 0000000..02edf2e
--- /dev/null
+++ b/hbase-metrics/src/main/resources/META-INF/services/org.apache.hadoop.hbase.metrics.MetricRegistries
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java
new file mode 100644
index 0000000..f0b4f8c
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestCounterImpl.java
@@ -0,0 +1,59 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link CounterImpl}.
+ */
+@Category(SmallTests.class)
+public class TestCounterImpl {
+
+  private Counter counter;
+
+  @Before public void setup() {
+    this.counter = new CounterImpl();
+  }
+
+  @Test public void testCounting() {
+    counter.increment();
+    assertEquals(1L, counter.getCount());
+    counter.increment();
+    assertEquals(2L, counter.getCount());
+    counter.increment(2L);
+    assertEquals(4L, counter.getCount());
+    counter.increment(-1L);
+    assertEquals(3L, counter.getCount());
+
+    counter.decrement();
+    assertEquals(2L, counter.getCount());
+    counter.decrement();
+    assertEquals(1L, counter.getCount());
+    counter.decrement(4L);
+    assertEquals(-3L, counter.getCount());
+    counter.decrement(-3L);
+    assertEquals(0L, counter.getCount());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
new file mode 100644
index 0000000..3ba9821
--- /dev/null
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
@@ -0,0 +1,51 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.codahale.metrics.Meter;
+
+/**
+ * Test class for {@link DropwizardMeter}.
+ */
+@Category(SmallTests.class)
+public class TestDropwizardMeter {
+
+  private Meter meter;
+
+  @Before public void setup() {
+    this.meter = Mockito.mock(Meter.class);
+  }
+
+  @Test public void test() {
+    DropwizardMeter dwMeter = new DropwizardMeter(this.meter);
+
+    dwMeter.mark();
+    dwMeter.mark(10L);
+    dwMeter.mark();
+    dwMeter.mark();
+
+    Mockito.verify(meter, Mockito.times(3)).mark();
+    Mockito.verify(meter).mark(10L);
+  }
+}
\ No newline at end of file