You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by dr...@apache.org on 2022/09/27 02:03:36 UTC

[ratis] branch master updated: RATIS-1712. Add a dropwizard 3 implementation of ratis-metrics-api. (#751)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c4448bfb3 RATIS-1712. Add a dropwizard 3 implementation of ratis-metrics-api. (#751)
c4448bfb3 is described below

commit c4448bfb3ec2032475cc105a19caa60c4d244632
Author: Tsz-Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Tue Sep 27 10:03:31 2022 +0800

    RATIS-1712. Add a dropwizard 3 implementation of ratis-metrics-api. (#751)
---
 pom.xml                                            |   6 +
 ratis-assembly/src/main/assembly/bin.xml           |   1 +
 ratis-assembly/src/main/assembly/src.xml           |   1 +
 .../ratis/metrics/TestMetricRegistriesLoader.java  |  21 ++-
 ratis-metrics-dropwizard3/pom.xml                  |  92 ++++++++++++
 .../ratis/metrics/dropwizard3/Dm3JvmMetrics.java   |  51 +++++++
 .../dropwizard3/Dm3MetricRegistriesImpl.java       | 129 +++++++++++++++++
 .../dropwizard3/Dm3MetricRegistryFactoryImpl.java  |  28 ++++
 .../metrics/dropwizard3/Dm3MetricsReporting.java   |  81 +++++++++++
 .../dropwizard3/Dm3RatisMetricRegistryImpl.java    | 157 +++++++++++++++++++++
 .../dropwizard3/Dm3RatisObjectNameFactory.java     |  44 ++++++
 .../metrics/dropwizard3/Dm3TimekeeperImpl.java     |  42 ++++++
 .../ratis/metrics/dropwizard3/RefCountingMap.java  |  94 ++++++++++++
 .../org.apache.ratis.metrics.MetricRegistries      |  18 +++
 .../dropwizard3/TestLoadDm3MetricRegistries.java   |  34 +++++
 .../metrics/dropwizard3/TestRefCountingMap.java    | 147 +++++++++++++++++++
 16 files changed, 939 insertions(+), 7 deletions(-)

diff --git a/pom.xml b/pom.xml
index aa2de1694..07e784879 100644
--- a/pom.xml
+++ b/pom.xml
@@ -85,6 +85,7 @@
     <module>ratis-replicated-map</module>
     <module>ratis-metrics-api</module>
     <module>ratis-metrics-default</module>
+    <module>ratis-metrics-dropwizard3</module>
     <module>ratis-tools</module>
     <module>ratis-shell</module>
     <module>ratis-assembly</module>
@@ -281,6 +282,11 @@
         <groupId>org.apache.ratis</groupId>
         <version>${project.version}</version>
       </dependency>
+      <dependency>
+        <artifactId>ratis-metrics-dropwizard3</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
       <dependency>
         <artifactId>ratis-metrics-default</artifactId>
         <groupId>org.apache.ratis</groupId>
diff --git a/ratis-assembly/src/main/assembly/bin.xml b/ratis-assembly/src/main/assembly/bin.xml
index 85c14fc33..f7e9cb517 100644
--- a/ratis-assembly/src/main/assembly/bin.xml
+++ b/ratis-assembly/src/main/assembly/bin.xml
@@ -41,6 +41,7 @@
         <include>org.apache.ratis:ratis-test</include>
         <include>org.apache.ratis:ratis-metrics-api</include>
         <include>org.apache.ratis:ratis-metrics-default</include>
+        <include>org.apache.ratis:ratis-metrics-dropwizard3</include>
         <include>org.apache.ratis:ratis-tools</include>
         <include>org.apache.ratis:ratis-resource-bundle</include>
       </includes>
diff --git a/ratis-assembly/src/main/assembly/src.xml b/ratis-assembly/src/main/assembly/src.xml
index 6b001af63..9559917a0 100644
--- a/ratis-assembly/src/main/assembly/src.xml
+++ b/ratis-assembly/src/main/assembly/src.xml
@@ -44,6 +44,7 @@
         <include>org.apache.ratis:ratis-test</include>
         <include>org.apache.ratis:ratis-metrics-api</include>
         <include>org.apache.ratis:ratis-metrics-default</include>
+        <include>org.apache.ratis:ratis-metrics-dropwizard3</include>
         <include>org.apache.ratis:ratis-tools</include>
         <include>org.apache.ratis:ratis-resource-bundle</include>
       </includes>
diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java
index cbfa6810c..d06dec8ff 100644
--- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java
+++ b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java
@@ -21,20 +21,21 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.mockito.Mockito.mock;
 
-import org.apache.ratis.thirdparty.com.google.common.collect.Lists;
+import org.apache.ratis.metrics.impl.MetricRegistriesImpl;
+import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Arrays;
+import java.util.Collections;
 
 /**
  * Test class for {@link MetricRegistriesLoader}.
  */
 public class TestMetricRegistriesLoader {
-
-
   @Test
-  public void testLoadSinleInstance() {
+  public void testLoadSingleInstance() {
     MetricRegistries loader = mock(MetricRegistries.class);
-    MetricRegistries instance = MetricRegistriesLoader.load(Lists.newArrayList(loader));
+    MetricRegistries instance = MetricRegistriesLoader.load(Collections.singletonList(loader));
     assertEquals(loader, instance);
   }
 
@@ -43,12 +44,18 @@ public class TestMetricRegistriesLoader {
     MetricRegistries loader1 = mock(MetricRegistries.class);
     MetricRegistries loader2 = mock(MetricRegistries.class);
     MetricRegistries loader3 = mock(MetricRegistries.class);
-    MetricRegistries instance = MetricRegistriesLoader.load(Lists.newArrayList(loader1, loader2,
-        loader3));
+    MetricRegistries instance = MetricRegistriesLoader.load(Arrays.asList(loader1, loader2, loader3));
 
     // the load() returns the first instance
     assertEquals(loader1, instance);
     assertNotEquals(loader2, instance);
     assertNotEquals(loader3, instance);
   }
+
+  @Test
+  public void testLoadDefault() {
+    final MetricRegistries r = MetricRegistriesLoader.load();
+    Assert.assertSame(MetricRegistriesImpl.class, r.getClass());
+  }
+
 }
diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml
new file mode 100644
index 000000000..fedabe9df
--- /dev/null
+++ b/ratis-metrics-dropwizard3/pom.xml
@@ -0,0 +1,92 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<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">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>3.0.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>ratis-metrics-dropwizard3</artifactId>
+  <name>Apache Ratis Metrics Dropwizard 3 Implementation</name>
+
+  <properties>
+    <dropwizard3.version>3.2.5</dropwizard3.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-metrics-api</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-proto</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <version>${dropwizard3.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-jvm</artifactId>
+      <version>${dropwizard3.version}</version>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-ganglia</artifactId>
+      <version>${dropwizard3.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.acplt</groupId>
+          <artifactId>oncrpc</artifactId>
+        </exclusion>
+      </exclusions>
+      <optional>true</optional>
+    </dependency>
+
+  </dependencies>
+</project>
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3JvmMetrics.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3JvmMetrics.java
new file mode 100644
index 000000000..e34c902d5
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3JvmMetrics.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis.metrics.dropwizard3;
+
+import org.apache.ratis.metrics.MetricRegistries;
+import org.apache.ratis.metrics.MetricRegistryInfo;
+import org.apache.ratis.metrics.RatisMetricRegistry;
+import org.apache.ratis.util.TimeDuration;
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+
+/**
+ * Helper class to add JVM metrics.
+ */
+public interface Dm3JvmMetrics {
+  static void initJvmMetrics(TimeDuration consoleReportRate) {
+    final MetricRegistries registries = MetricRegistries.global();
+    Dm3JvmMetrics.addJvmMetrics(registries);
+    registries.enableConsoleReporter(consoleReportRate);
+    registries.enableJmxReporter();
+  }
+
+  static void addJvmMetrics(MetricRegistries registries) {
+    MetricRegistryInfo info = new MetricRegistryInfo("jvm", "ratis_jvm", "jvm", "jvm metrics");
+
+    RatisMetricRegistry registry = registries.create(info);
+
+    final Dm3RatisMetricRegistryImpl impl = Dm3RatisMetricRegistryImpl.cast(registry);
+    impl.registerAll("gc", new GarbageCollectorMetricSet());
+    impl.registerAll("memory", new MemoryUsageGaugeSet());
+    impl.registerAll("threads", new ThreadStatesGaugeSet());
+    impl.registerAll("classLoading", new ClassLoadingGaugeSet());
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java
new file mode 100644
index 000000000..7fe4b8daa
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java
@@ -0,0 +1,129 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis.metrics.dropwizard3;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Consumer;
+
+import org.apache.ratis.metrics.MetricRegistries;
+import org.apache.ratis.metrics.MetricRegistryInfo;
+import org.apache.ratis.metrics.RatisMetricRegistry;
+import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of MetricRegistries that does ref-counting.
+ */
+public class Dm3MetricRegistriesImpl extends MetricRegistries {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Dm3MetricRegistriesImpl.class);
+
+  private final List<Consumer<RatisMetricRegistry>> reporterRegistrations = new CopyOnWriteArrayList<>();
+
+  private final List<Consumer<RatisMetricRegistry>> stopReporters = new CopyOnWriteArrayList<>();
+
+  private final Dm3MetricRegistryFactoryImpl factory;
+
+  private final RefCountingMap<MetricRegistryInfo, RatisMetricRegistry> registries;
+
+  public Dm3MetricRegistriesImpl() {
+    this(new Dm3MetricRegistryFactoryImpl());
+  }
+
+  Dm3MetricRegistriesImpl(Dm3MetricRegistryFactoryImpl factory) {
+    this.factory = factory;
+    this.registries = new RefCountingMap<>();
+  }
+
+  @Override
+  public RatisMetricRegistry create(MetricRegistryInfo info) {
+    return registries.put(info, () -> {
+      if (reporterRegistrations.isEmpty()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("First MetricRegistry has been created without registering reporters. " +
+              "Hence registering JMX reporter by default.");
+        }
+        enableJmxReporter();
+      }
+      RatisMetricRegistry registry = factory.create(info);
+      reporterRegistrations.forEach(reg -> reg.accept(registry));
+      return registry;
+    });
+  }
+
+  @Override
+  public boolean remove(MetricRegistryInfo key) {
+    RatisMetricRegistry registry = registries.get(key);
+    if (registry != null) {
+      stopReporters.forEach(reg -> reg.accept(registry));
+    }
+
+    return registries.remove(key) == null;
+  }
+
+  @Override
+  public Optional<RatisMetricRegistry> get(MetricRegistryInfo info) {
+    return Optional.ofNullable(registries.get(info));
+  }
+
+  @Override
+  public Collection<RatisMetricRegistry> getMetricRegistries() {
+    return Collections.unmodifiableCollection(registries.values());
+  }
+
+  @Override
+  public void clear() {
+    registries.clear();
+  }
+
+  @Override
+  public Set<MetricRegistryInfo> getMetricRegistryInfos() {
+    return Collections.unmodifiableSet(registries.keySet());
+  }
+
+  @Override
+  public void addReporterRegistration(Consumer<RatisMetricRegistry> reporterRegistration,
+      Consumer<RatisMetricRegistry> stopReporter) {
+    if (registries.size() > 0) {
+      LOG.warn("New reporters are added after registries were created. Some metrics will be missing from the reporter. "
+          + "Please add reporter before adding any new registry.");
+    }
+    this.reporterRegistrations.add(reporterRegistration);
+    this.stopReporters.add(stopReporter);
+  }
+
+  @Override
+  public void enableJmxReporter() {
+    addReporterRegistration(
+        Dm3MetricsReporting.jmxReporter(),
+        Dm3MetricsReporting.stopJmxReporter());
+  }
+
+  @Override
+  public void enableConsoleReporter(TimeDuration consoleReportRate) {
+    addReporterRegistration(
+        Dm3MetricsReporting.consoleReporter(consoleReportRate),
+        Dm3MetricsReporting.stopConsoleReporter());
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistryFactoryImpl.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistryFactoryImpl.java
new file mode 100644
index 000000000..32497c01f
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistryFactoryImpl.java
@@ -0,0 +1,28 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis.metrics.dropwizard3;
+
+import org.apache.ratis.metrics.MetricRegistryFactory;
+import org.apache.ratis.metrics.MetricRegistryInfo;
+
+public class Dm3MetricRegistryFactoryImpl implements MetricRegistryFactory {
+  @Override
+  public Dm3RatisMetricRegistryImpl create(MetricRegistryInfo info) {
+    return new Dm3RatisMetricRegistryImpl(info);
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricsReporting.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricsReporting.java
new file mode 100644
index 000000000..d4ffb34e7
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricsReporting.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis.metrics.dropwizard3;
+
+import org.apache.ratis.metrics.RatisMetricRegistry;
+import org.apache.ratis.util.TimeDuration;
+
+import com.codahale.metrics.ConsoleReporter;
+import com.codahale.metrics.ScheduledReporter;
+import com.codahale.metrics.JmxReporter;
+
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+final class Dm3MetricsReporting {
+  private Dm3MetricsReporting() {
+  }
+
+  static Consumer<RatisMetricRegistry> consoleReporter(TimeDuration rate) {
+    return registry -> consoleReporter(rate, registry);
+  }
+
+  private static void consoleReporter(TimeDuration rate, RatisMetricRegistry registry) {
+    final Dm3RatisMetricRegistryImpl impl = Dm3RatisMetricRegistryImpl.cast(registry);
+    final ConsoleReporter reporter = ConsoleReporter.forRegistry(impl.getDropWizardMetricRegistry())
+        .convertRatesTo(TimeUnit.SECONDS)
+        .convertDurationsTo(TimeUnit.MILLISECONDS)
+        .build();
+    reporter.start(rate.getDuration(), rate.getUnit());
+    impl.setConsoleReporter(reporter);
+  }
+
+  static Consumer<RatisMetricRegistry> stopConsoleReporter() {
+    return Dm3MetricsReporting::stopConsoleReporter;
+  }
+
+  private static void stopConsoleReporter(RatisMetricRegistry registry) {
+    final Dm3RatisMetricRegistryImpl impl = Dm3RatisMetricRegistryImpl.cast(registry);
+    Optional.ofNullable(impl.getConsoleReporter()).ifPresent(ScheduledReporter::close);
+  }
+
+  static Consumer<RatisMetricRegistry> jmxReporter() {
+    return Dm3MetricsReporting::jmxReporter;
+  }
+
+  private static void jmxReporter(RatisMetricRegistry registry) {
+    final Dm3RatisMetricRegistryImpl impl = Dm3RatisMetricRegistryImpl.cast(registry);
+    final JmxReporter reporter = JmxReporter.forRegistry(impl.getDropWizardMetricRegistry())
+        .inDomain(registry.getMetricRegistryInfo().getApplicationName())
+        .createsObjectNamesWith(new Dm3RatisObjectNameFactory())
+        .build();
+    reporter.start();
+    impl.setJmxReporter(reporter);
+  }
+
+
+  static Consumer<RatisMetricRegistry> stopJmxReporter() {
+    return Dm3MetricsReporting::stopJmxReporter;
+  }
+
+  private static void stopJmxReporter(RatisMetricRegistry registry) {
+    final Dm3RatisMetricRegistryImpl impl = Dm3RatisMetricRegistryImpl.cast(registry);
+    Optional.ofNullable(impl.getJmxReporter()).ifPresent(JmxReporter::close);
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3RatisMetricRegistryImpl.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3RatisMetricRegistryImpl.java
new file mode 100644
index 000000000..d8b20d108
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3RatisMetricRegistryImpl.java
@@ -0,0 +1,157 @@
+/*
+ * 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.ratis.metrics.dropwizard3;
+
+import org.apache.ratis.metrics.LongCounter;
+import org.apache.ratis.metrics.MetricRegistryInfo;
+import org.apache.ratis.metrics.RatisMetricRegistry;
+import org.apache.ratis.metrics.Timekeeper;
+import com.codahale.metrics.ConsoleReporter;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.JmxReporter;
+import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.function.Supplier;
+
+/**
+ * Custom implementation of {@link MetricRegistry}.
+ */
+public class Dm3RatisMetricRegistryImpl implements RatisMetricRegistry {
+  static Dm3RatisMetricRegistryImpl cast(RatisMetricRegistry registry) {
+    if (!(registry instanceof Dm3RatisMetricRegistryImpl)) {
+      throw new IllegalStateException("Unexpected class: " + registry.getClass().getName());
+    }
+    return (Dm3RatisMetricRegistryImpl) registry;
+  }
+
+  private final MetricRegistry metricRegistry = new MetricRegistry();
+
+  private final MetricRegistryInfo info;
+  private final String namePrefix;
+
+  private JmxReporter jmxReporter;
+  private ConsoleReporter consoleReporter;
+
+  public Dm3RatisMetricRegistryImpl(MetricRegistryInfo info) {
+    this.info = Objects.requireNonNull(info, "info == null");
+    this.namePrefix = MetricRegistry.name(info.getApplicationName(), info.getMetricsComponentName(), info.getPrefix());
+  }
+
+  @Override
+  public Timekeeper timer(String name) {
+    return new Dm3TimekeeperImpl(metricRegistry.timer(getMetricName(name)));
+  }
+
+  static LongCounter toLongCounter(Counter c) {
+    return new LongCounter() {
+      @Override
+      public void inc(long n) {
+        c.inc(n);
+      }
+
+      @Override
+      public void dec(long n) {
+        c.dec(n);
+      }
+
+      @Override
+      public long getCount() {
+        return c.getCount();
+      }
+    };
+  }
+
+  @Override
+  public LongCounter counter(String name) {
+    return toLongCounter(metricRegistry.counter(getMetricName(name)));
+  }
+
+  @Override
+  public boolean remove(String name) {
+    return metricRegistry.remove(getMetricName(name));
+  }
+
+  static <T> Gauge<T> toGauge(Supplier<T> supplier) {
+    return supplier::get;
+  }
+
+  @Override
+  public <T> void gauge(String name, Supplier<Supplier<T>> gaugeSupplier) {
+    metricRegistry.gauge(getMetricName(name), () -> toGauge(gaugeSupplier.get()));
+  }
+
+  public SortedMap<String, Gauge> getGauges(MetricFilter filter) {
+    return metricRegistry.getGauges(filter);
+  }
+
+  @VisibleForTesting
+  public Metric get(String shortName) {
+    return metricRegistry.getMetrics().get(getMetricName(shortName));
+  }
+
+  private String getMetricName(String shortName) {
+    return MetricRegistry.name(namePrefix, shortName);
+  }
+
+  private <T extends Metric> T register(String name, T metric) throws IllegalArgumentException {
+    return metricRegistry.register(getMetricName(name), metric);
+  }
+
+
+  public MetricRegistry getDropWizardMetricRegistry() {
+    return metricRegistry;
+  }
+
+  @Override public MetricRegistryInfo getMetricRegistryInfo(){
+    return this.info;
+  }
+
+  void registerAll(String prefix, MetricSet metricSet) {
+    for (Map.Entry<String, Metric> entry : metricSet.getMetrics().entrySet()) {
+      if (entry.getValue() instanceof MetricSet) {
+        registerAll(prefix + "." + entry.getKey(), (MetricSet) entry.getValue());
+      } else {
+        register(prefix + "." + entry.getKey(), entry.getValue());
+      }
+    }
+  }
+
+  void setJmxReporter(JmxReporter jmxReporter) {
+    this.jmxReporter = jmxReporter;
+  }
+
+  JmxReporter getJmxReporter() {
+    return this.jmxReporter;
+  }
+
+  void setConsoleReporter(ConsoleReporter consoleReporter) {
+    this.consoleReporter = consoleReporter;
+  }
+
+  ConsoleReporter getConsoleReporter() {
+    return this.consoleReporter;
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3RatisObjectNameFactory.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3RatisObjectNameFactory.java
new file mode 100644
index 000000000..7f6c21930
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3RatisObjectNameFactory.java
@@ -0,0 +1,44 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis.metrics.dropwizard3;
+
+import com.codahale.metrics.ObjectNameFactory;
+
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+class Dm3RatisObjectNameFactory implements ObjectNameFactory {
+  @Override
+  public ObjectName createName(String type, String domain, String name) {
+    try {
+      ObjectName objectName = new ObjectName(domain, "name", name);
+      if (objectName.isPattern()) {
+        objectName = new ObjectName(domain, "name", ObjectName.quote(name));
+      }
+      return objectName;
+    } catch (MalformedObjectNameException e) {
+      try {
+        return new ObjectName(domain, "name", ObjectName.quote(name));
+      } catch (MalformedObjectNameException mone) {
+        throw new IllegalArgumentException(
+            "Failed to register " + name + ", type=" + type + ", domain=" + domain, mone);
+      }
+    }
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3TimekeeperImpl.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3TimekeeperImpl.java
new file mode 100644
index 000000000..8dbcb2944
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3TimekeeperImpl.java
@@ -0,0 +1,42 @@
+/*
+ * 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.ratis.metrics.dropwizard3;
+
+import org.apache.ratis.metrics.Timekeeper;
+import com.codahale.metrics.Timer;
+
+/**
+ * The default implementation of {@link Timekeeper} by the shaded {@link Timer}.
+ */
+public class Dm3TimekeeperImpl implements Timekeeper {
+  private final Timer timer;
+
+  Dm3TimekeeperImpl(Timer timer) {
+    this.timer = timer;
+  }
+
+  public Timer getTimer() {
+    return timer;
+  }
+
+  @Override
+  public Context time() {
+    final Timer.Context context = timer.time();
+    return context::stop;
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java
new file mode 100644
index 000000000..6c3ad6f8c
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java
@@ -0,0 +1,94 @@
+/*
+ * 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.ratis.metrics.dropwizard3;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+/**
+ * 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.
+ */
+class RefCountingMap<K, V> {
+  private static class Payload<V> {
+    private final V value;
+    private final AtomicInteger refCount = new AtomicInteger();
+
+    Payload(V v) {
+      this.value = v;
+    }
+
+    V get() {
+      return value;
+    }
+
+    V increment() {
+      return refCount.incrementAndGet() > 0? value: null;
+    }
+
+    Payload<V> decrement() {
+      return refCount.decrementAndGet() > 0? this: null;
+    }
+  }
+
+  private final ConcurrentMap<K, Payload<V>> map = new ConcurrentHashMap<>();
+
+  V put(K k, Supplier<V> supplier) {
+    return map.compute(k, (k1, old) -> old != null? old: new Payload<>(supplier.get())).increment();
+  }
+
+  static <V> V get(Payload<V> p) {
+    return p == null ? null : p.get();
+  }
+
+  V get(K k) {
+    return get(map.get(k));
+  }
+
+  /**
+   * 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) {
+    return get(map.computeIfPresent(k, (k1, v) -> v.decrement()));
+  }
+
+  void clear() {
+    map.clear();
+  }
+
+  Set<K> keySet() {
+    return map.keySet();
+  }
+
+  Collection<V> values() {
+    return map.values().stream().map(Payload::get).collect(Collectors.toList());
+  }
+
+  int size() {
+    return map.size();
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries b/ratis-metrics-dropwizard3/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries
new file mode 100644
index 000000000..a4c32318c
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/main/resources/META-INF/services/org.apache.ratis.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.ratis.metrics.dropwizard3.Dm3MetricRegistriesImpl
diff --git a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.java b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.java
new file mode 100644
index 000000000..dce4e2b94
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestLoadDm3MetricRegistries.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.ratis.metrics.dropwizard3;
+
+import org.apache.ratis.metrics.MetricRegistries;
+import org.apache.ratis.metrics.MetricRegistriesLoader;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class for {@link MetricRegistriesLoader}.
+ */
+public class TestLoadDm3MetricRegistries {
+  @Test
+  public void testLoadDm3() {
+    final MetricRegistries r = MetricRegistriesLoader.load();
+    Assert.assertSame(Dm3MetricRegistriesImpl.class, r.getClass());
+  }
+}
diff --git a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java
new file mode 100644
index 000000000..1aa15d37c
--- /dev/null
+++ b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java
@@ -0,0 +1,147 @@
+/*
+ * 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.ratis.metrics.dropwizard3;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collection;
+import java.util.Set;
+
+import org.apache.ratis.thirdparty.com.google.common.collect.Lists;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRefCountingMap {
+
+  private RefCountingMap<String, String> map;
+
+  @Before
+  public void setUp() {
+    map = new RefCountingMap<>();
+  }
+
+  @Test
+  public void testPutGet() {
+    map.put("foo", () -> "foovalue");
+
+    String v = map.get("foo");
+    assertNotNull(v);
+    assertEquals("foovalue", v);
+  }
+
+  @Test
+  public void testPutMulti() {
+    String v1 = map.put("foo", () -> "foovalue");
+    String v2 =  map.put("foo", () -> "foovalue2");
+    String v3 = map.put("foo", () -> "foovalue3");
+
+    String v = map.get("foo");
+    assertEquals("foovalue", v);
+    assertEquals(v, v1);
+    assertEquals(v, v2);
+    assertEquals(v, v3);
+  }
+
+  @Test
+  public void testPutRemove() {
+    map.put("foo", () -> "foovalue");
+    String v = map.remove("foo");
+    assertNull(v);
+    v = map.get("foo");
+    assertNull(v);
+  }
+
+  @Test
+  public void testPutRemoveMulti() {
+    map.put("foo", () -> "foovalue");
+    map.put("foo", () -> "foovalue2");
+    map.put("foo", () -> "foovalue3");
+
+    // remove 1
+    String v = map.remove("foo");
+    assertEquals("foovalue", v);
+
+    // remove 2
+    v = map.remove("foo");
+    assertEquals("foovalue", v);
+
+    // remove 3
+    v = map.remove("foo");
+    assertNull(v);
+    v = map.get("foo");
+    assertNull(v);
+  }
+
+  @Test
+  public void testSize() {
+    assertEquals(0, map.size());
+
+    // put a key
+    map.put("foo", () -> "foovalue");
+    assertEquals(1, map.size());
+
+    // put a different key
+    map.put("bar", () -> "foovalue2");
+    assertEquals(2, map.size());
+
+    // put the same key again
+    map.put("bar", () -> "foovalue3");
+    assertEquals(2, map.size()); // map should be same size
+  }
+
+  @Test
+  public void testClear() {
+    map.put("foo", () -> "foovalue");
+    map.put("bar", () -> "foovalue2");
+    map.put("baz", () -> "foovalue3");
+
+    map.clear();
+
+    assertEquals(0, map.size());
+  }
+
+
+  @Test
+  public void testKeySet() {
+    map.put("foo", () -> "foovalue");
+    map.put("bar", () -> "foovalue2");
+    map.put("baz", () -> "foovalue3");
+
+    Set<String> keys = map.keySet();
+    assertEquals(3, keys.size());
+
+    Lists.newArrayList("foo", "bar", "baz").stream().forEach(v -> assertTrue(keys.contains(v)));
+  }
+
+  @Test
+  public void testValues() {
+    map.put("foo", () -> "foovalue");
+    map.put("foo", () -> "foovalue2");
+    map.put("bar", () -> "foovalue3");
+    map.put("baz", () -> "foovalue4");
+
+    Collection<String> values = map.values();
+    assertEquals(3, values.size());
+
+    Lists.newArrayList("foovalue", "foovalue3", "foovalue4").stream()
+            .forEach(v -> assertTrue(values.contains(v)));
+  }
+}