You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2019/10/17 13:26:17 UTC

[lucene-solr] branch jira/solr-13677-final updated: SOLR-13677: Cleanup & refactor getMetricRegistry().

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

ab pushed a commit to branch jira/solr-13677-final
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/solr-13677-final by this push:
     new 8287d06  SOLR-13677: Cleanup & refactor getMetricRegistry().
8287d06 is described below

commit 8287d060c3395bb0911c98860a15a2bc7e0fed3e
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Oct 17 15:17:19 2019 +0200

    SOLR-13677: Cleanup & refactor getMetricRegistry().
---
 .../java/org/apache/solr/core/SolrInfoBean.java    |  6 +++
 .../apache/solr/handler/RequestHandlerBase.java    |  9 +---
 .../java/org/apache/solr/search/CaffeineCache.java |  6 ---
 .../java/org/apache/solr/search/FastLRUCache.java  |  5 --
 .../src/java/org/apache/solr/search/LFUCache.java  |  6 ---
 .../src/java/org/apache/solr/search/LRUCache.java  |  5 --
 .../org/apache/solr/search/SolrFieldCacheBean.java |  5 --
 .../org/apache/solr/search/SolrIndexSearcher.java  |  5 --
 .../apache/solr/security/AuditLoggerPlugin.java    | 45 ++++++++--------
 .../apache/solr/security/AuthenticationPlugin.java |  9 +---
 .../apache/solr/update/DirectUpdateHandler2.java   | 61 ++++++++++++----------
 .../java/org/apache/solr/update/UpdateHandler.java |  5 --
 .../org/apache/solr/update/UpdateShardHandler.java | 21 +++++---
 13 files changed, 75 insertions(+), 113 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
index bfb3428..dc0f599 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoBean.java
@@ -21,6 +21,8 @@ import java.util.Set;
 
 import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricsContext;
 import org.apache.solr.util.stats.MetricUtils;
 
 /**
@@ -77,6 +79,10 @@ public interface SolrInfoBean {
    * (default is null, which means no registry).
    */
   default MetricRegistry getMetricRegistry() {
+    if (this instanceof SolrMetricProducer) {
+      SolrMetricsContext context = ((SolrMetricProducer)this).getSolrMetricsContext();
+      return context != null ? context.getMetricRegistry() : null;
+    }
     return null;
   }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index b744abf..8af5391 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import com.google.common.collect.ImmutableList;
 import org.apache.solr.api.Api;
@@ -79,6 +78,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   private PluginInfo pluginInfo;
 
   private Set<String> metricNames = ConcurrentHashMap.newKeySet();
+  protected SolrMetricsContext solrMetricsContext;
 
 
   @SuppressForbidden(reason = "Need currentTimeMillis, used only for stats output")
@@ -140,8 +140,6 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
 
   }
 
-  protected SolrMetricsContext solrMetricsContext;
-
   @Override
   public SolrMetricsContext getSolrMetricsContext() {
     return solrMetricsContext;
@@ -275,11 +273,6 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext != null ? solrMetricsContext.getMetricRegistry() : null;
-  }
-
-  @Override
   public SolrRequestHandler getSubHandler(String subPath) {
     return null;
   }
diff --git a/solr/core/src/java/org/apache/solr/search/CaffeineCache.java b/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
index 1fb5902..9472701 100644
--- a/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
+++ b/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
@@ -31,7 +31,6 @@ import java.util.concurrent.ForkJoinPool;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.LongAdder;
 
-import com.codahale.metrics.MetricRegistry;
 import com.github.benmanes.caffeine.cache.RemovalCause;
 import com.github.benmanes.caffeine.cache.RemovalListener;
 import org.apache.lucene.util.Accountable;
@@ -323,11 +322,6 @@ public class CaffeineCache<K, V> extends SolrCacheBase implements SolrCache<K, V
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext != null ? solrMetricsContext.getMetricRegistry() : null;
-  }
-
-  @Override
   public SolrMetricsContext getSolrMetricsContext() {
     return solrMetricsContext;
   }
diff --git a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
index 35b8dd1..d538fa7 100644
--- a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
@@ -327,11 +327,6 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K, V>
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext == null ? null : solrMetricsContext.getMetricRegistry();
-  }
-
-  @Override
   public String toString() {
     return name() + (cacheMap != null ? cacheMap.getValue().toString() : "");
   }
diff --git a/solr/core/src/java/org/apache/solr/search/LFUCache.java b/solr/core/src/java/org/apache/solr/search/LFUCache.java
index f3800da..96f87c9 100644
--- a/solr/core/src/java/org/apache/solr/search/LFUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LFUCache.java
@@ -351,12 +351,6 @@ public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext == null ? null : solrMetricsContext.getMetricRegistry();
-
-  }
-
-  @Override
   public String toString() {
     return name + (cacheMap != null ? cacheMap.getValue().toString() : "");
   }
diff --git a/solr/core/src/java/org/apache/solr/search/LRUCache.java b/solr/core/src/java/org/apache/solr/search/LRUCache.java
index 5755fdb..86c2ed6 100644
--- a/solr/core/src/java/org/apache/solr/search/LRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LRUCache.java
@@ -441,11 +441,6 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext ==null ?null: solrMetricsContext.getMetricRegistry();
-  }
-
-  @Override
   public String toString() {
     return name() + (cacheMap != null ? cacheMap.getValue().toString() : "");
   }
diff --git a/solr/core/src/java/org/apache/solr/search/SolrFieldCacheBean.java b/solr/core/src/java/org/apache/solr/search/SolrFieldCacheBean.java
index e266caa..d43c1d5 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrFieldCacheBean.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrFieldCacheBean.java
@@ -19,7 +19,6 @@ package org.apache.solr.search;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
-import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricProducer;
@@ -50,10 +49,6 @@ public class SolrFieldCacheBean implements SolrInfoBean, SolrMetricProducer {
   public Set<String> getMetricNames() {
     return metricNames;
   }
-  @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext != null ? solrMetricsContext.getMetricRegistry() : null;
-  }
 
   @Override
   public SolrMetricsContext getSolrMetricsContext() {
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 8c7954e..27bf584 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -2319,11 +2319,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         }), true, "statsCache", Category.CACHE.toString(), scope);
   }
 
-  @Override
-  public MetricRegistry getMetricRegistry() {
-    return core.getMetricRegistry();
-  }
-
   private static class FilterImpl extends Filter {
     private final Filter topFilter;
     private final List<Weight> weights;
diff --git a/solr/core/src/java/org/apache/solr/security/AuditLoggerPlugin.java b/solr/core/src/java/org/apache/solr/security/AuditLoggerPlugin.java
index a6c364a..395102a 100644
--- a/solr/core/src/java/org/apache/solr/security/AuditLoggerPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuditLoggerPlugin.java
@@ -36,7 +36,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -45,8 +44,8 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.core.SolrInfoBean;
-import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricsContext;
 import org.apache.solr.security.AuditEvent.EventType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,14 +74,12 @@ public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfo
   int blockingQueueSize;
 
   protected AuditEventFormatter formatter;
-  private MetricRegistry registry;
   private Set<String> metricNames = ConcurrentHashMap.newKeySet();
   private ExecutorService executorService;
   private boolean closed;
   private MuteRules muteRules;
-  
-  protected String registryName;
-  protected SolrMetricManager metricManager;
+
+  protected SolrMetricsContext solrMetricsContext;
   protected Meter numErrors = new Meter();
   protected Meter numLost = new Meter();
   protected Meter numLogged = new Meter();
@@ -239,24 +236,21 @@ public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfo
   }
   
   @Override
-  public void initializeMetrics(SolrMetricManager manager, String registryName, String tag, final String scope) {
+  public void initializeMetrics(SolrMetricsContext m, final String scope) {
+    solrMetricsContext = m.getChildContext(this);
     String className = this.getClass().getSimpleName();
     log.debug("Initializing metrics for {}", className);
-    this.metricManager = manager;
-    this.registryName = registryName;
-    // Metrics
-    registry = manager.registry(registryName);
-    numErrors = manager.meter(this, registryName, "errors", getCategory().toString(), scope, className);
-    numLost = manager.meter(this, registryName, "lost", getCategory().toString(), scope, className);
-    numLogged = manager.meter(this, registryName, "count", getCategory().toString(), scope, className);
-    requestTimes = manager.timer(this, registryName, "requestTimes", getCategory().toString(), scope, className);
-    totalTime = manager.counter(this, registryName, "totalTime", getCategory().toString(), scope, className);
+    numErrors = solrMetricsContext.meter(this, "errors", getCategory().toString(), scope, className);
+    numLost = solrMetricsContext.meter(this, "lost", getCategory().toString(), scope, className);
+    numLogged = solrMetricsContext.meter(this, "count", getCategory().toString(), scope, className);
+    requestTimes = solrMetricsContext.timer(this, "requestTimes", getCategory().toString(), scope, className);
+    totalTime = solrMetricsContext.counter(this, "totalTime", getCategory().toString(), scope, className);
     if (async) {
-      manager.registerGauge(this, registryName, () -> blockingQueueSize, "queueCapacity", true, "queueCapacity", getCategory().toString(), scope, className);
-      manager.registerGauge(this, registryName, () -> blockingQueueSize - queue.remainingCapacity(), "queueSize", true, "queueSize", getCategory().toString(), scope, className);
-      queuedTime = manager.timer(this, registryName, "queuedTime", getCategory().toString(), scope, className);
+      solrMetricsContext.gauge(this, () -> blockingQueueSize, true, "queueCapacity", getCategory().toString(), scope, className);
+      solrMetricsContext.gauge(this, () -> blockingQueueSize - queue.remainingCapacity(), true, "queueSize", getCategory().toString(), scope, className);
+      queuedTime = solrMetricsContext.timer(this, "queuedTime", getCategory().toString(), scope, className);
     }
-    manager.registerGauge(this, registryName, () -> async, "async", true, "async", getCategory().toString(), scope, className);
+    solrMetricsContext.gauge(this, () -> async, true, "async", getCategory().toString(), scope, className);
   }
   
   @Override
@@ -280,10 +274,10 @@ public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfo
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return registry;
+  public SolrMetricsContext getSolrMetricsContext() {
+    return solrMetricsContext;
   }
-  
+
   /**
    * Interface for formatting the event
    */
@@ -325,6 +319,11 @@ public abstract class AuditLoggerPlugin implements Closeable, Runnable, SolrInfo
       closed = true;
       log.info("Shutting down async Auditlogger background thread(s)");
       executorService.shutdownNow();
+      try {
+        SolrMetricProducer.super.close();
+      } catch (Exception e) {
+        throw new IOException("Exception closing", e);
+      }
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index c118694..eda4938 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import org.apache.http.HttpRequest;
 import org.apache.http.protocol.HttpContext;
@@ -45,6 +44,7 @@ public abstract class AuthenticationPlugin implements SolrInfoBean, SolrMetricPr
 
   // Metrics
   private Set<String> metricNames = ConcurrentHashMap.newKeySet();
+  protected SolrMetricsContext solrMetricsContext;
 
   protected Meter numErrors = new Meter();
   protected Counter requests = new Counter();
@@ -137,7 +137,6 @@ public abstract class AuthenticationPlugin implements SolrInfoBean, SolrMetricPr
    */
   public void closeRequest() {
   }
-  protected SolrMetricsContext solrMetricsContext;
 
   @Override
   public SolrMetricsContext getSolrMetricsContext() {
@@ -177,10 +176,4 @@ public abstract class AuthenticationPlugin implements SolrInfoBean, SolrMetricPr
   public Set<String> getMetricNames() {
     return metricNames;
   }
-
-  @Override
-  public MetricRegistry getMetricRegistry() {
-    return solrMetricsContext == null ? null : solrMetricsContext.getMetricRegistry();
-  }
-
 }
diff --git a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
index 2b621a8..3d4702e 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -53,6 +53,7 @@ import org.apache.solr.core.SolrConfig.UpdateHandlerInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricsContext;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
@@ -96,8 +97,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   LongAdder numDocsPending = new LongAdder();
   LongAdder numErrors = new LongAdder();
   Meter numErrorsCumulative;
-  SolrMetricManager metricManager;
-  String registryName;
+  SolrMetricsContext solrMetricsContext;
 
   // tracks when auto-commit should occur
   protected final CommitTracker commitTracker;
@@ -170,48 +170,46 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   }
 
   @Override
-  public void initializeMetrics(SolrMetricManager manager, String registryName, String tag, String scope) {
-    this.metricManager = manager;
-    this.registryName = registryName;
-    this.registry = manager.registry(registryName);
-    commitCommands = manager.meter(this, registryName, "commits", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> commitTracker.getCommitCount(), tag, true, "autoCommits", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> softCommitTracker.getCommitCount(), tag, true, "softAutoCommits", getCategory().toString(), scope);
+  public void initializeMetrics(SolrMetricsContext m, String scope) {
+    solrMetricsContext = m.getChildContext(this);
+    commitCommands = solrMetricsContext.meter(this, "commits", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> commitTracker.getCommitCount(), true, "autoCommits", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> softCommitTracker.getCommitCount(), true, "softAutoCommits", getCategory().toString(), scope);
     if (commitTracker.getDocsUpperBound() > 0) {
-      manager.registerGauge(this, registryName, () -> commitTracker.getDocsUpperBound(), tag, true, "autoCommitMaxDocs",
+      solrMetricsContext.gauge(this, () -> commitTracker.getDocsUpperBound(), true, "autoCommitMaxDocs",
           getCategory().toString(), scope);
     }
     if (commitTracker.getTimeUpperBound() > 0) {
-      manager.registerGauge(this, registryName, () -> "" + commitTracker.getTimeUpperBound() + "ms", tag, true, "autoCommitMaxTime",
+      solrMetricsContext.gauge(this, () -> "" + commitTracker.getTimeUpperBound() + "ms", true, "autoCommitMaxTime",
           getCategory().toString(), scope);
     }
     if (commitTracker.getTLogFileSizeUpperBound() > 0) {
-      manager.registerGauge(this, registryName, () -> commitTracker.getTLogFileSizeUpperBound(), tag, true, "autoCommitMaxSize",
+      solrMetricsContext.gauge(this, () -> commitTracker.getTLogFileSizeUpperBound(), true, "autoCommitMaxSize",
           getCategory().toString(), scope);
     }
     if (softCommitTracker.getDocsUpperBound() > 0) {
-      manager.registerGauge(this, registryName, () -> softCommitTracker.getDocsUpperBound(), tag, true, "softAutoCommitMaxDocs",
+      solrMetricsContext.gauge(this, () -> softCommitTracker.getDocsUpperBound(), true, "softAutoCommitMaxDocs",
           getCategory().toString(), scope);
     }
     if (softCommitTracker.getTimeUpperBound() > 0) {
-      manager.registerGauge(this, registryName, () -> "" + softCommitTracker.getTimeUpperBound() + "ms", tag, true, "softAutoCommitMaxTime",
+      solrMetricsContext.gauge(this, () -> "" + softCommitTracker.getTimeUpperBound() + "ms", true, "softAutoCommitMaxTime",
           getCategory().toString(), scope);
     }
-    optimizeCommands = manager.meter(this, registryName, "optimizes", getCategory().toString(), scope);
-    rollbackCommands = manager.meter(this, registryName, "rollbacks", getCategory().toString(), scope);
-    splitCommands = manager.meter(this, registryName, "splits", getCategory().toString(), scope);
-    mergeIndexesCommands = manager.meter(this, registryName, "merges", getCategory().toString(), scope);
-    expungeDeleteCommands = manager.meter(this, registryName, "expungeDeletes", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> numDocsPending.longValue(), tag, true, "docsPending", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> addCommands.longValue(), tag, true, "adds", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> deleteByIdCommands.longValue(), tag, true, "deletesById", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> deleteByQueryCommands.longValue(), tag, true, "deletesByQuery", getCategory().toString(), scope);
-    manager.registerGauge(this, registryName, () -> numErrors.longValue(), tag, true, "errors", getCategory().toString(), scope);
+    optimizeCommands = solrMetricsContext.meter(this, "optimizes", getCategory().toString(), scope);
+    rollbackCommands = solrMetricsContext.meter(this, "rollbacks", getCategory().toString(), scope);
+    splitCommands = solrMetricsContext.meter(this, "splits", getCategory().toString(), scope);
+    mergeIndexesCommands = solrMetricsContext.meter(this, "merges", getCategory().toString(), scope);
+    expungeDeleteCommands = solrMetricsContext.meter(this, "expungeDeletes", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> numDocsPending.longValue(), true, "docsPending", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> addCommands.longValue(), true, "adds", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> deleteByIdCommands.longValue(), true, "deletesById", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> deleteByQueryCommands.longValue(), true, "deletesByQuery", getCategory().toString(), scope);
+    solrMetricsContext.gauge(this, () -> numErrors.longValue(), true, "errors", getCategory().toString(), scope);
 
-    addCommandsCumulative = manager.meter(this, registryName, "cumulativeAdds", getCategory().toString(), scope);
-    deleteByIdCommandsCumulative = manager.meter(this, registryName, "cumulativeDeletesById", getCategory().toString(), scope);
-    deleteByQueryCommandsCumulative = manager.meter(this, registryName, "cumulativeDeletesByQuery", getCategory().toString(), scope);
-    numErrorsCumulative = manager.meter(this, registryName, "cumulativeErrors", getCategory().toString(), scope);
+    addCommandsCumulative = solrMetricsContext.meter(this, "cumulativeAdds", getCategory().toString(), scope);
+    deleteByIdCommandsCumulative = solrMetricsContext.meter(this, "cumulativeDeletesById", getCategory().toString(), scope);
+    deleteByQueryCommandsCumulative = solrMetricsContext.meter(this, "cumulativeDeletesByQuery", getCategory().toString(), scope);
+    numErrorsCumulative = solrMetricsContext.meter(this, "cumulativeErrors", getCategory().toString(), scope);
   }
 
   private void deleteAll() throws IOException {
@@ -805,6 +803,11 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
     softCommitTracker.close();
 
     numDocsPending.reset();
+    try {
+      SolrMetricProducer.super.close();
+    } catch (Exception e) {
+      throw new IOException("Error closing", e);
+    }
   }
 
 
@@ -915,7 +918,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
   }
 
   /**
-   * Calls either {@link IndexWriter#updateDocValues} or {@link IndexWriter#updateDocument}(s) as
+   * Calls either {@link IndexWriter#updateDocValues} or <code>IndexWriter#updateDocument</code>(s) as
    * needed based on {@link AddUpdateCommand#isInPlaceUpdate}.
    * <p>
    * If the this is an UPDATE_INPLACE cmd, then all fields included in 
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
index c8dbc10..cec5d37 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
@@ -57,7 +57,6 @@ public abstract class UpdateHandler implements SolrInfoBean {
   protected final UpdateLog ulog;
 
   protected Set<String> metricNames = ConcurrentHashMap.newKeySet();
-  protected MetricRegistry registry;
 
   private void parseEventListeners() {
     final Class<SolrEventListener> clazz = SolrEventListener.class;
@@ -211,8 +210,4 @@ public abstract class UpdateHandler implements SolrInfoBean {
   public Set<String> getMetricNames() {
     return metricNames;
   }
-  @Override
-  public MetricRegistry getMetricRegistry() {
-    return registry;
-  }
 }
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index f41102c..3d32abb 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -91,7 +91,7 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
 
 
   private final Set<String> metricNames = ConcurrentHashMap.newKeySet();
-  private MetricRegistry registry;
+  private SolrMetricsContext solrMetricsContext;
 
   private int socketTimeout = HttpClientUtil.DEFAULT_SO_TIMEOUT;
   private int connectionTimeout = HttpClientUtil.DEFAULT_CONNECT_TIMEOUT;
@@ -181,13 +181,13 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
 
   @Override
   public void initializeMetrics(SolrMetricsContext m, String scope) {
-    registry = m.getMetricRegistry();
+    solrMetricsContext = m.getChildContext(this);
     String expandedScope = SolrMetricManager.mkName(scope, getCategory().name());
-    updateHttpListenerFactory.initializeMetrics(m, expandedScope);
-    defaultConnectionManager.initializeMetrics(m, expandedScope);
-    updateExecutor = MetricUtils.instrumentedExecutorService(updateExecutor, this, registry,
+    updateHttpListenerFactory.initializeMetrics(solrMetricsContext, expandedScope);
+    defaultConnectionManager.initializeMetrics(solrMetricsContext, expandedScope);
+    updateExecutor = MetricUtils.instrumentedExecutorService(updateExecutor, this, solrMetricsContext.getMetricRegistry(),
         SolrMetricManager.mkName("updateOnlyExecutor", expandedScope, "threadPool"));
-    recoveryExecutor = MetricUtils.instrumentedExecutorService(recoveryExecutor, this, registry,
+    recoveryExecutor = MetricUtils.instrumentedExecutorService(recoveryExecutor, this, solrMetricsContext.getMetricRegistry(),
         SolrMetricManager.mkName("recoveryExecutor", expandedScope, "threadPool"));
   }
 
@@ -207,8 +207,8 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
   }
 
   @Override
-  public MetricRegistry getMetricRegistry() {
-    return registry;
+  public SolrMetricsContext getSolrMetricsContext() {
+    return solrMetricsContext;
   }
 
   // if you are looking for a client to use, it's probably this one.
@@ -260,6 +260,11 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
     } catch (Exception e) {
       throw new RuntimeException(e);
     } finally {
+      try {
+        SolrMetricProducer.super.close();
+      } catch (Exception e) {
+        // do nothing
+      }
       IOUtils.closeQuietly(updateOnlyClient);
       HttpClientUtil.close(recoveryOnlyClient);
       HttpClientUtil.close(defaultClient);