You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ta...@apache.org on 2023/02/24 02:30:18 UTC

[iotdb] branch master updated: [IOTDB-5559] Implement metric exporters for RatisConsensus (#9099)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 753d3f29fb [IOTDB-5559] Implement metric exporters for RatisConsensus (#9099)
753d3f29fb is described below

commit 753d3f29fb1b4275d6f315b2c48c980ad9edb7dd
Author: William Song <48...@users.noreply.github.com>
AuthorDate: Fri Feb 24 10:30:12 2023 +0800

    [IOTDB-5559] Implement metric exporters for RatisConsensus (#9099)
---
 .../consensus/ratis/metrics/CounterProxy.java      |  57 ++++++
 .../iotdb/consensus/ratis/metrics/GaugeProxy.java  |  45 +++++
 .../ratis/metrics/IoTDBMetricRegistry.java         | 202 +++++++++++++++++++++
 .../ratis/metrics/MetricRegistryManager.java       |  98 ++++++++++
 .../consensus/ratis/metrics/RatisMetricSet.java    |  38 ++++
 .../consensus/ratis/metrics/RefCountingMap.java    |  97 ++++++++++
 .../iotdb/consensus/ratis/metrics/TimerProxy.java  |  42 +++++
 .../org.apache.ratis.metrics.MetricRegistries      |  17 ++
 8 files changed, 596 insertions(+)

diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/CounterProxy.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/CounterProxy.java
new file mode 100644
index 0000000000..9211c858b2
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/CounterProxy.java
@@ -0,0 +1,57 @@
+/*
+ * 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.iotdb.consensus.ratis.metrics;
+
+import org.apache.iotdb.metrics.type.Counter;
+
+/** A Proxy class using IoTDB Counter to replace the dropwizard Counter. */
+public class CounterProxy extends com.codahale.metrics.Counter {
+
+  /** IoTDB Counter */
+  private final Counter counter;
+
+  CounterProxy(Counter counter) {
+    this.counter = counter;
+  }
+
+  @Override
+  public void inc() {
+    inc(1L);
+  }
+
+  @Override
+  public void inc(long n) {
+    counter.inc(n);
+  }
+
+  @Override
+  public void dec() {
+    inc(-1L);
+  }
+
+  @Override
+  public void dec(long n) {
+    inc(-n);
+  }
+
+  @Override
+  public long getCount() {
+    return counter.count();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/GaugeProxy.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/GaugeProxy.java
new file mode 100644
index 0000000000..b790fb5f59
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/GaugeProxy.java
@@ -0,0 +1,45 @@
+/*
+ * 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.iotdb.consensus.ratis.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricRegistry;
+
+/** AutoGauge supplier holder class */
+public class GaugeProxy implements Gauge {
+
+  private final Gauge gauge;
+
+  public GaugeProxy(MetricRegistry.MetricSupplier<Gauge> metricSupplier) {
+    this.gauge = metricSupplier.newMetric();
+  }
+
+  @Override
+  public Object getValue() {
+    return gauge.getValue();
+  }
+
+  double getValueAsDouble() {
+    Object value = getValue();
+    if (value instanceof Number) {
+      return ((Number) value).doubleValue();
+    }
+    return 0.0;
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java
new file mode 100644
index 0000000000..4c54504ed7
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/IoTDBMetricRegistry.java
@@ -0,0 +1,202 @@
+/*
+ * 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.iotdb.consensus.ratis.metrics;
+
+import org.apache.iotdb.metrics.AbstractMetricService;
+import org.apache.iotdb.metrics.utils.MetricLevel;
+import org.apache.iotdb.metrics.utils.MetricType;
+
+import com.codahale.metrics.ConsoleReporter;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.Timer;
+import org.apache.ratis.metrics.MetricRegistryInfo;
+import org.apache.ratis.metrics.RatisMetricRegistry;
+
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class IoTDBMetricRegistry implements RatisMetricRegistry {
+
+  private final AbstractMetricService metricService;
+  private final MetricRegistryInfo info;
+  private final String prefix;
+  private final Map<String, String> metricNameCache = new ConcurrentHashMap<>();
+  private final Map<String, CounterProxy> counterCache = new ConcurrentHashMap<>();
+  private final Map<String, TimerProxy> timerCache = new ConcurrentHashMap<>();
+  private final Map<String, GaugeProxy> gaugeCache = new ConcurrentHashMap<>();
+
+  IoTDBMetricRegistry(MetricRegistryInfo info, AbstractMetricService service) {
+    this.info = info;
+    this.metricService = service;
+    prefix =
+        MetricRegistry.name(
+            info.getApplicationName(), info.getMetricsComponentName(), info.getPrefix());
+  }
+
+  private String getMetricName(String name) {
+    return metricNameCache.computeIfAbsent(name, n -> MetricRegistry.name(prefix, n));
+  }
+
+  @Override
+  public Timer timer(String name) {
+    final String fullName = getMetricName(name);
+    return timerCache.computeIfAbsent(
+        fullName, fn -> new TimerProxy(metricService.getOrCreateTimer(fn, MetricLevel.IMPORTANT)));
+  }
+
+  @Override
+  public Counter counter(String name) {
+    final String fullName = getMetricName(name);
+    return counterCache.computeIfAbsent(
+        fullName,
+        fn ->
+            new CounterProxy(
+                metricService.getOrCreateCounter(getMetricName(name), MetricLevel.IMPORTANT)));
+  }
+
+  @Override
+  public boolean remove(String name) {
+    // Currently MetricService in IoTDB does not support to remove a metric by its name only.
+    // Therefore, we are trying every potential type here util we remove it successfully.
+    // Since metricService.remove will throw an IllegalArgument when type mismatches, so use three
+    // independent try-clauses
+    // TODO (szywilliam) we can add an interface like removeTypeless(name)
+    try {
+      metricService.remove(MetricType.COUNTER, getMetricName(name));
+    } catch (IllegalArgumentException ignored) {
+    }
+    try {
+      metricService.remove(MetricType.TIMER, getMetricName(name));
+    } catch (IllegalArgumentException ignored) {
+    }
+    try {
+      metricService.remove(MetricType.AUTO_GAUGE, getMetricName(name));
+    } catch (IllegalArgumentException ignored) {
+    }
+
+    return true;
+  }
+
+  void removeAll() {
+    counterCache.forEach((name, counter) -> metricService.remove(MetricType.COUNTER, name));
+    gaugeCache.forEach((name, gauge) -> metricService.remove(MetricType.AUTO_GAUGE, name));
+    timerCache.forEach((name, timer) -> metricService.remove(MetricType.TIMER, name));
+    metricNameCache.clear();
+    counterCache.clear();
+    gaugeCache.clear();
+    timerCache.clear();
+  }
+
+  @Override
+  public Gauge gauge(String name, MetricRegistry.MetricSupplier<Gauge> metricSupplier) {
+    final String fullName = getMetricName(name);
+    return gaugeCache.computeIfAbsent(
+        fullName,
+        gaugeName -> {
+          final GaugeProxy gauge = new GaugeProxy(metricSupplier);
+          metricService.createAutoGauge(
+              gaugeName, MetricLevel.IMPORTANT, gauge, GaugeProxy::getValueAsDouble);
+          return gauge;
+        });
+  }
+
+  @Override
+  public Timer timer(String name, MetricRegistry.MetricSupplier<Timer> metricSupplier) {
+    throw new UnsupportedOperationException("This method is not used in IoTDB project");
+  }
+
+  @Override
+  public SortedMap<String, Gauge> getGauges(MetricFilter metricFilter) {
+    throw new UnsupportedOperationException("This method is not used in IoTDB project");
+  }
+
+  @Override
+  public Counter counter(String name, MetricRegistry.MetricSupplier<Counter> metricSupplier) {
+    throw new UnsupportedOperationException("This method is not used in IoTDB project");
+  }
+
+  @Override
+  public Histogram histogram(String name) {
+    throw new UnsupportedOperationException("Histogram is not used in Ratis Metrics");
+  }
+
+  @Override
+  public Meter meter(String name) {
+    throw new UnsupportedOperationException("Meter is not used in Ratis Metrics");
+  }
+
+  @Override
+  public Meter meter(String name, MetricRegistry.MetricSupplier<Meter> metricSupplier) {
+    throw new UnsupportedOperationException("Meter is not used in Ratis Metrics");
+  }
+
+  @Override
+  public Metric get(String name) {
+    throw new UnsupportedOperationException("Meter is not used in Ratis Metrics");
+  }
+
+  @Override
+  public <T extends Metric> T register(String name, T t) throws IllegalArgumentException {
+    throw new UnsupportedOperationException("register is not used in Ratis Metrics");
+  }
+
+  @Override
+  public MetricRegistry getDropWizardMetricRegistry() {
+    throw new UnsupportedOperationException("This method is not used in IoTDB project");
+  }
+
+  @Override
+  public MetricRegistryInfo getMetricRegistryInfo() {
+    return info;
+  }
+
+  @Override
+  public void registerAll(String s, MetricSet metricSet) {
+    throw new UnsupportedOperationException("registerAll is not used in Ratis Metrics");
+  }
+
+  @Override
+  public void setJmxReporter(JmxReporter jmxReporter) {
+    throw new UnsupportedOperationException("JmxReporter is not used in Ratis Metrics");
+  }
+
+  @Override
+  public JmxReporter getJmxReporter() {
+    throw new UnsupportedOperationException("JmxReporter is not used in Ratis Metrics");
+  }
+
+  @Override
+  public void setConsoleReporter(ConsoleReporter consoleReporter) {
+    throw new UnsupportedOperationException("ConsoleReporter is not used in Ratis Metrics");
+  }
+
+  @Override
+  public ConsoleReporter getConsoleReporter() {
+    throw new UnsupportedOperationException("ConsoleReporter is not used in Ratis Metrics");
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java
new file mode 100644
index 0000000000..d0d5f0b4da
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/MetricRegistryManager.java
@@ -0,0 +1,98 @@
+/*
+ * 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.iotdb.consensus.ratis.metrics;
+
+import org.apache.iotdb.metrics.AbstractMetricService;
+import org.apache.iotdb.metrics.DoNothingMetricService;
+
+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 java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Consumer;
+
+public class MetricRegistryManager extends MetricRegistries {
+  // Using RefCountingMap here because of potential duplicate MetricRegistryInfos
+  private final RefCountingMap<MetricRegistryInfo, RatisMetricRegistry> registries;
+  // TODO: enable ratis metrics after verifying its correctness and efficiency
+  private final AbstractMetricService service = new DoNothingMetricService();
+
+  public MetricRegistryManager() {
+    this.registries = new RefCountingMap<>();
+  }
+
+  @Override
+  public void clear() {
+    registries.values().stream()
+        .map(registry -> (IoTDBMetricRegistry) registry)
+        .forEach(IoTDBMetricRegistry::removeAll);
+    this.registries.clear();
+  }
+
+  @Override
+  public RatisMetricRegistry create(MetricRegistryInfo metricRegistryInfo) {
+    return registries.put(
+        metricRegistryInfo, () -> new IoTDBMetricRegistry(metricRegistryInfo, service));
+  }
+
+  @Override
+  public boolean remove(MetricRegistryInfo metricRegistryInfo) {
+    return registries.remove(metricRegistryInfo) == null;
+  }
+
+  @Override
+  public Optional<RatisMetricRegistry> get(MetricRegistryInfo metricRegistryInfo) {
+    return Optional.ofNullable(registries.get(metricRegistryInfo));
+  }
+
+  @Override
+  public Set<MetricRegistryInfo> getMetricRegistryInfos() {
+    return Collections.unmodifiableSet(registries.keySet());
+  }
+
+  @Override
+  public Collection<RatisMetricRegistry> getMetricRegistries() {
+    return Collections.unmodifiableCollection(registries.values());
+  }
+
+  @Override
+  public void addReporterRegistration(
+      Consumer<RatisMetricRegistry> reporterRegistration,
+      Consumer<RatisMetricRegistry> stopReporter) {
+    throw new UnsupportedOperationException("Reporter is disabled from RatisMetricRegistries");
+  }
+
+  @Override
+  public void enableJmxReporter() {
+    // We shall disable the JMX reporter since we already have one in MetricService
+    throw new UnsupportedOperationException("JMX Reporter is disabled from RatisMetricRegistries");
+  }
+
+  @Override
+  public void enableConsoleReporter(TimeDuration timeDuration) {
+    // We shall disable the Console reporter since we already have one in MetricService
+    throw new UnsupportedOperationException(
+        "Console Reporter is disabled from RatisMetricRegistries");
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java
new file mode 100644
index 0000000000..a376490ec7
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RatisMetricSet.java
@@ -0,0 +1,38 @@
+/*
+ * 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.iotdb.consensus.ratis.metrics;
+
+import org.apache.iotdb.metrics.AbstractMetricService;
+import org.apache.iotdb.metrics.metricsets.IMetricSet;
+
+import org.apache.ratis.metrics.MetricRegistries;
+
+public class RatisMetricSet implements IMetricSet {
+  private MetricRegistries manager;
+
+  @Override
+  public void bindTo(AbstractMetricService metricService) {
+    manager = MetricRegistries.global();
+  }
+
+  @Override
+  public void unbindFrom(AbstractMetricService metricService) {
+    manager.clear();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RefCountingMap.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RefCountingMap.java
new file mode 100644
index 0000000000..e5f63d1459
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/RefCountingMap.java
@@ -0,0 +1,97 @@
+/*
+ * 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.iotdb.consensus.ratis.metrics;
+
+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;
+    }
+
+    RefCountingMap.Payload<V> decrement() {
+      return refCount.decrementAndGet() > 0 ? this : null;
+    }
+  }
+
+  private final ConcurrentMap<K, RefCountingMap.Payload<V>> map = new ConcurrentHashMap<>();
+
+  V put(K k, Supplier<V> supplier) {
+    return map.compute(
+            k, (k1, old) -> old != null ? old : new RefCountingMap.Payload<>(supplier.get()))
+        .increment();
+  }
+
+  static <V> V get(RefCountingMap.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(RefCountingMap.Payload::get).collect(Collectors.toList());
+  }
+
+  int size() {
+    return map.size();
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/TimerProxy.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/TimerProxy.java
new file mode 100644
index 0000000000..c5f16e8e76
--- /dev/null
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/metrics/TimerProxy.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.iotdb.consensus.ratis.metrics;
+
+import org.apache.iotdb.metrics.type.Timer;
+
+import java.util.concurrent.TimeUnit;
+
+public class TimerProxy extends com.codahale.metrics.Timer {
+  private final Timer timer;
+
+  TimerProxy(Timer timer) {
+    this.timer = timer;
+  }
+
+  @Override
+  // time() method is used as a user time clock. Will reuse the dropwizard implementation.
+  public Context time() {
+    return super.time();
+  }
+
+  @Override
+  public void update(long duration, TimeUnit unit) {
+    timer.update(duration, unit);
+  }
+}
diff --git a/consensus/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries b/consensus/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries
new file mode 100644
index 0000000000..dfb0de5189
--- /dev/null
+++ b/consensus/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries
@@ -0,0 +1,17 @@
+#
+# 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.iotdb.consensus.ratis.metrics.MetricRegistryManager
\ No newline at end of file