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 2017/04/11 17:39:11 UTC

[5/7] lucene-solr:master: SOLR-9959: SolrInfoMBean-s category and hierarchy cleanup.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/MetricsMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/MetricsMap.java b/solr/core/src/java/org/apache/solr/metrics/MetricsMap.java
new file mode 100644
index 0000000..f43c60b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/MetricsMap.java
@@ -0,0 +1,184 @@
+/*
+ * 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.solr.metrics;
+
+import javax.management.Attribute;
+import javax.management.AttributeList;
+import javax.management.AttributeNotFoundException;
+import javax.management.DynamicMBean;
+import javax.management.InvalidAttributeValueException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.ReflectionException;
+import javax.management.openmbean.OpenMBeanAttributeInfoSupport;
+import javax.management.openmbean.OpenType;
+import javax.management.openmbean.SimpleType;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.BiConsumer;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.solr.common.SolrException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dynamically constructed map of metrics, intentionally different from {@link com.codahale.metrics.MetricSet}
+ * where each metric had to be known in advance and registered separately in {@link com.codahale.metrics.MetricRegistry}.
+ * <p>Note: this awkwardly extends {@link Gauge} and not {@link Metric} because awkwardly {@link Metric} instances
+ * are not supported by {@link com.codahale.metrics.MetricRegistryListener} :(</p>
+ * <p>Note 2: values added to this metric map should belong to the list of types supported by JMX:
+ * {@link javax.management.openmbean.OpenType#ALLOWED_CLASSNAMES_LIST}, otherwise only their toString()
+ * representation will be shown in JConsole.</p>
+ */
+public class MetricsMap implements Gauge<Map<String,Object>>, DynamicMBean {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  // set to true to use cached statistics between getMBeanInfo calls to work
+  // around over calling getStatistics on MBeanInfos when iterating over all attributes (SOLR-6586)
+  private final boolean useCachedStatsBetweenGetMBeanInfoCalls = Boolean.getBoolean("useCachedStatsBetweenGetMBeanInfoCalls");
+
+  private BiConsumer<Boolean, Map<String, Object>> initializer;
+  private volatile Map<String,Object> cachedValue;
+
+  public MetricsMap(BiConsumer<Boolean, Map<String,Object>> initializer) {
+    this.initializer = initializer;
+  }
+
+  @Override
+  public Map<String,Object> getValue() {
+    return getValue(true);
+  }
+
+  public Map<String,Object> getValue(boolean detailed) {
+    Map<String,Object> map = new HashMap<>();
+    initializer.accept(detailed, map);
+    return map;
+  }
+
+  public String toString() {
+    return getValue().toString();
+  }
+
+  @Override
+  public Object getAttribute(String attribute) throws AttributeNotFoundException, MBeanException, ReflectionException {
+    Object val;
+    Map<String,Object> stats = null;
+    if (useCachedStatsBetweenGetMBeanInfoCalls) {
+      Map<String,Object> cachedStats = this.cachedValue;
+      if (cachedStats != null) {
+        stats = cachedStats;
+      }
+    }
+    if (stats == null) {
+      stats = getValue(true);
+    }
+    val = stats.get(attribute);
+
+    if (val != null) {
+      // It's String or one of the simple types, just return it as JMX suggests direct support for such types
+      for (String simpleTypeName : SimpleType.ALLOWED_CLASSNAMES_LIST) {
+        if (val.getClass().getName().equals(simpleTypeName)) {
+          return val;
+        }
+      }
+      // It's an arbitrary object which could be something complex and odd, return its toString, assuming that is
+      // a workable representation of the object
+      return val.toString();
+    }
+    return null;
+  }
+
+  @Override
+  public void setAttribute(Attribute attribute) throws AttributeNotFoundException, InvalidAttributeValueException, MBeanException, ReflectionException {
+    throw new UnsupportedOperationException("Operation not Supported");
+  }
+
+  @Override
+  public AttributeList getAttributes(String[] attributes) {
+    AttributeList list = new AttributeList();
+    for (String attribute : attributes) {
+      try {
+        list.add(new Attribute(attribute, getAttribute(attribute)));
+      } catch (Exception e) {
+        log.warn("Could not get attribute " + attribute);
+      }
+    }
+    return list;
+  }
+
+  @Override
+  public AttributeList setAttributes(AttributeList attributes) {
+    throw new UnsupportedOperationException("Operation not Supported");
+  }
+
+  @Override
+  public Object invoke(String actionName, Object[] params, String[] signature) throws MBeanException, ReflectionException {
+    throw new UnsupportedOperationException("Operation not Supported");
+  }
+
+  @Override
+  public MBeanInfo getMBeanInfo() {
+    ArrayList<MBeanAttributeInfo> attrInfoList = new ArrayList<>();
+    Map<String,Object> stats = getValue(true);
+    if (useCachedStatsBetweenGetMBeanInfoCalls) {
+      cachedValue = stats;
+    }
+    try {
+      stats.forEach((k, v) -> {
+        Class type = v.getClass();
+        OpenType typeBox = determineType(type);
+        if (type.equals(String.class) || typeBox == null) {
+          attrInfoList.add(new MBeanAttributeInfo(k, String.class.getName(),
+              null, true, false, false));
+        } else {
+          attrInfoList.add(new OpenMBeanAttributeInfoSupport(
+              k, k, typeBox, true, false, false));
+        }
+      });
+    } catch (Exception e) {
+      // don't log issue if the core is closing
+      if (!(SolrException.getRootCause(e) instanceof AlreadyClosedException))
+        log.warn("Could not get attributes of MetricsMap: {}", this, e);
+    }
+    MBeanAttributeInfo[] attrInfoArr = attrInfoList
+        .toArray(new MBeanAttributeInfo[attrInfoList.size()]);
+    return new MBeanInfo(getClass().getName(), "MetricsMap", attrInfoArr, null, null, null);
+  }
+
+  private OpenType determineType(Class type) {
+    try {
+      for (Field field : SimpleType.class.getFields()) {
+        if (field.getType().equals(SimpleType.class)) {
+          SimpleType candidate = (SimpleType) field.get(SimpleType.class);
+          if (candidate.getTypeName().equals(type.getName())) {
+            return candidate;
+          }
+        }
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/OperatingSystemMetricSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/OperatingSystemMetricSet.java b/solr/core/src/java/org/apache/solr/metrics/OperatingSystemMetricSet.java
index 34ef5d1..21957eb 100644
--- a/solr/core/src/java/org/apache/solr/metrics/OperatingSystemMetricSet.java
+++ b/solr/core/src/java/org/apache/solr/metrics/OperatingSystemMetricSet.java
@@ -16,77 +16,31 @@
  */
 package org.apache.solr.metrics;
 
-import javax.management.JMException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 
-import com.codahale.metrics.JmxAttributeGauge;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.solr.util.stats.MetricUtils;
 
 /**
  * This is an extended replacement for {@link com.codahale.metrics.jvm.FileDescriptorRatioGauge}
- * - that class uses reflection and doesn't work under Java 9. We can also get much more
- * information about OS environment once we have to go through MBeanServer anyway.
+ * - that class uses reflection and doesn't work under Java 9. This implementation tries to retrieve
+ * bean properties from known implementations of {@link java.lang.management.OperatingSystemMXBean}.
  */
 public class OperatingSystemMetricSet implements MetricSet {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /** Metric names - these correspond to known numeric MBean attributes. Depending on the OS and
-   * Java implementation only some of them may be actually present.
-   */
-  public static final String[] METRICS = {
-      "AvailableProcessors",
-      "CommittedVirtualMemorySize",
-      "FreePhysicalMemorySize",
-      "FreeSwapSpaceSize",
-      "MaxFileDescriptorCount",
-      "OpenFileDescriptorCount",
-      "ProcessCpuLoad",
-      "ProcessCpuTime",
-      "SystemLoadAverage",
-      "TotalPhysicalMemorySize",
-      "TotalSwapSpaceSize"
-  };
-
-  private final MBeanServer mBeanServer;
-
-  public OperatingSystemMetricSet(MBeanServer mBeanServer) {
-    this.mBeanServer = mBeanServer;
-  }
 
   @Override
   public Map<String, Metric> getMetrics() {
     final Map<String, Metric> metrics = new HashMap<>();
-
-    try {
-      final ObjectName on = new ObjectName("java.lang:type=OperatingSystem");
-      // verify that it exists
-      MBeanInfo info = mBeanServer.getMBeanInfo(on);
-      // collect valid attributes
-      Set<String> attributes = new HashSet<>();
-      for (MBeanAttributeInfo ai : info.getAttributes()) {
-        attributes.add(ai.getName());
-      }
-      for (String metric : METRICS) {
-        // verify that an attribute exists before attempting to add it
-        if (attributes.contains(metric)) {
-          metrics.put(metric, new JmxAttributeGauge(mBeanServer, on, metric));
-        }
+    OperatingSystemMXBean os = ManagementFactory.getOperatingSystemMXBean();
+    MetricUtils.addMXBeanMetrics(os, MetricUtils.OS_MXBEAN_CLASSES, null, (k, v) -> {
+      if (!metrics.containsKey(k)) {
+        metrics.put(k, v);
       }
-    } catch (JMException ignored) {
-      log.debug("Unable to load OperatingSystem MBean", ignored);
-    }
-
+    });
     return metrics;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
index 43f3535..193bf68 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -20,11 +20,12 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrInfoBean;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -76,14 +77,14 @@ public class SolrCoreMetricManager implements Closeable {
   }
 
   /**
-   * Load reporters configured globally and specific to {@link org.apache.solr.core.SolrInfoMBean.Group#core}
+   * Load reporters configured globally and specific to {@link org.apache.solr.core.SolrInfoBean.Group#core}
    * group or with a registry name specific to this core.
    */
   public void loadReporters() {
     NodeConfig nodeConfig = core.getCoreDescriptor().getCoreContainer().getConfig();
     PluginInfo[] pluginInfos = nodeConfig.getMetricReporterPlugins();
     metricManager.loadReporters(pluginInfos, core.getResourceLoader(), tag,
-        SolrInfoMBean.Group.core, registryName);
+        SolrInfoBean.Group.core, registryName);
     if (cloudMode) {
       metricManager.loadShardReporters(pluginInfos, core);
     }
@@ -127,11 +128,25 @@ public class SolrCoreMetricManager implements Closeable {
   }
 
   /**
+   * Return the registry used by this SolrCore.
+   */
+  public MetricRegistry getRegistry() {
+    if (registryName != null) {
+      return metricManager.registry(registryName);
+    } else {
+      return null;
+    }
+  }
+
+  /**
    * Closes reporters specific to this core.
    */
   @Override
   public void close() throws IOException {
     metricManager.closeReporters(getRegistryName(), tag);
+    if (getLeaderRegistryName() != null) {
+      metricManager.closeReporters(getLeaderRegistryName(), tag);
+    }
   }
 
   public SolrCore getCore() {
@@ -176,9 +191,9 @@ public class SolrCoreMetricManager implements Closeable {
 
   public static String createRegistryName(boolean cloud, String collectionName, String shardName, String replicaName, String coreName) {
     if (cloud) { // build registry name from logical names
-      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, collectionName, shardName, replicaName);
+      return SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, collectionName, shardName, replicaName);
     } else {
-      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, coreName);
+      return SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, coreName);
     }
   }
 
@@ -224,7 +239,7 @@ public class SolrCoreMetricManager implements Closeable {
 
   public static String createLeaderRegistryName(boolean cloud, String collectionName, String shardName) {
     if (cloud) {
-      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.collection, collectionName, shardName, "leader");
+      return SolrMetricManager.getRegistryName(SolrInfoBean.Group.collection, collectionName, shardName, "leader");
     } else {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
index 4d093eb..8edfa04 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
@@ -17,7 +17,7 @@
 package org.apache.solr.metrics;
 
 import com.codahale.metrics.MetricRegistry;
-import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrInfoBean;
 
 /**
  * Wraps meta-data for a metric.
@@ -25,7 +25,7 @@ import org.apache.solr.core.SolrInfoMBean;
 public final class SolrMetricInfo {
   public final String name;
   public final String scope;
-  public final SolrInfoMBean.Category category;
+  public final SolrInfoBean.Category category;
 
   /**
    * Creates a new instance of {@link SolrMetricInfo}.
@@ -34,7 +34,7 @@ public final class SolrMetricInfo {
    * @param scope    the scope of the metric (e.g. `/admin/ping`)
    * @param name     the name of the metric (e.g. `Requests`)
    */
-  public SolrMetricInfo(SolrInfoMBean.Category category, String scope, String name) {
+  public SolrMetricInfo(SolrInfoBean.Category category, String scope, String name) {
     this.name = name;
     this.scope = scope;
     this.category = category;
@@ -45,18 +45,25 @@ public final class SolrMetricInfo {
       return null;
     }
     String[] names = fullName.split("\\.");
-    if (names.length < 3) { // not a valid info
+    if (names.length < 2) { // not a valid info
       return null;
     }
     // check top-level name for valid category
-    SolrInfoMBean.Category category;
+    SolrInfoBean.Category category;
     try {
-      category = SolrInfoMBean.Category.valueOf(names[0]);
+      category = SolrInfoBean.Category.valueOf(names[0]);
     } catch (IllegalArgumentException e) { // not a valid category
       return null;
     }
-    String scope = names[1];
-    String name = fullName.substring(names[0].length() + names[1].length() + 2);
+    String scope;
+    String name;
+    if (names.length == 2) {
+      scope = null;
+      name = fullName.substring(names[0].length() + 1);
+    } else {
+      scope = names[1];
+      name = fullName.substring(names[0].length() + names[1].length() + 2);
+    }
     return new SolrMetricInfo(category, scope, name);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
index f4abee0..d4eb06a 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -51,7 +51,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.metrics.reporters.solr.SolrClusterReporter;
 import org.apache.solr.metrics.reporters.solr.SolrShardReporter;
@@ -69,11 +69,11 @@ import org.slf4j.LoggerFactory;
  * {@link MetricRegistry} instances are automatically created when first referenced by name. Similarly,
  * instances of {@link Metric} implementations, such as {@link Meter}, {@link Counter}, {@link Timer} and
  * {@link Histogram} are automatically created and registered under hierarchical names, in a specified
- * registry, when {@link #meter(String, String, String...)} and other similar methods are called.
+ * registry, when {@link #meter(SolrInfoBean, String, String, String...)} and other similar methods are called.
  * <p>This class enforces a common prefix ({@link #REGISTRY_NAME_PREFIX}) in all registry
  * names.</p>
  * <p>Solr uses several different registries for collecting metrics belonging to different groups, using
- * {@link org.apache.solr.core.SolrInfoMBean.Group} as the main name of the registry (plus the
+ * {@link org.apache.solr.core.SolrInfoBean.Group} as the main name of the registry (plus the
  * above-mentioned prefix). Instances of {@link SolrMetricManager} are created for each {@link org.apache.solr.core.CoreContainer},
  * and most registries are local to each instance, with the exception of two global registries:
  * <code>solr.jetty</code> and <code>solr.jvm</code>, which are shared between all {@link org.apache.solr.core.CoreContainer}-s</p>
@@ -87,11 +87,11 @@ public class SolrMetricManager {
 
   /** Registry name for Jetty-specific metrics. This name is also subject to overrides controlled by
    * system properties. This registry is shared between instances of {@link SolrMetricManager}. */
-  public static final String JETTY_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jetty.toString();
+  public static final String JETTY_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoBean.Group.jetty.toString();
 
   /** Registry name for JVM-specific metrics. This name is also subject to overrides controlled by
    * system properties. This registry is shared between instances of {@link SolrMetricManager}. */
-  public static final String JVM_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jvm.toString();
+  public static final String JVM_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoBean.Group.jvm.toString();
 
   private final ConcurrentMap<String, MetricRegistry> registries = new ConcurrentHashMap<>();
 
@@ -247,6 +247,66 @@ public class SolrMetricManager {
     }
   }
 
+  public static class OrFilter implements MetricFilter {
+    List<MetricFilter> filters = new ArrayList<>();
+
+    public OrFilter(Collection<MetricFilter> filters) {
+      if (filters != null) {
+        this.filters.addAll(filters);
+      }
+    }
+
+    public OrFilter(MetricFilter... filters) {
+      if (filters != null) {
+        for (MetricFilter filter : filters) {
+          if (filter != null) {
+            this.filters.add(filter);
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean matches(String s, Metric metric) {
+      for (MetricFilter filter : filters) {
+        if (filter.matches(s, metric)) {
+          return true;
+        }
+      }
+      return false;
+    }
+  }
+
+  public static class AndFilter implements MetricFilter {
+    List<MetricFilter> filters = new ArrayList<>();
+
+    public AndFilter(Collection<MetricFilter> filters) {
+      if (filters != null) {
+        this.filters.addAll(filters);
+      }
+    }
+
+    public AndFilter(MetricFilter... filters) {
+      if (filters != null) {
+        for (MetricFilter filter : filters) {
+          if (filter != null) {
+            this.filters.add(filter);
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean matches(String s, Metric metric) {
+      for (MetricFilter filter : filters) {
+        if (!filter.matches(s, metric)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
   /**
    * Return a set of existing registry names.
    */
@@ -452,6 +512,21 @@ public class SolrMetricManager {
   }
 
   /**
+   * Retrieve matching metrics and their names.
+   * @param registry registry name.
+   * @param metricFilter filter (null is equivalent to {@link MetricFilter#ALL}).
+   * @return map of matching names and metrics
+   */
+  public Map<String, Metric> getMetrics(String registry, MetricFilter metricFilter) {
+    if (metricFilter == null || metricFilter == MetricFilter.ALL) {
+      return registry(registry).getMetrics();
+    }
+    return registry(registry).getMetrics().entrySet().stream()
+        .filter(entry -> metricFilter.matches(entry.getKey(), entry.getValue()))
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> entry.getValue()));
+  }
+
+  /**
    * Create or get an existing named {@link Meter}
    * @param registry registry name
    * @param metricName metric name, either final name or a fully-qualified name
@@ -459,8 +534,12 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Meter}
    */
-  public Meter meter(String registry, String metricName, String... metricPath) {
-    return registry(registry).meter(mkName(metricName, metricPath));
+  public Meter meter(SolrInfoBean info, String registry, String metricName, String... metricPath) {
+    final String name = mkName(metricName, metricPath);
+    if (info != null) {
+      info.registerMetricName(name);
+    }
+    return registry(registry).meter(name);
   }
 
   /**
@@ -471,8 +550,12 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Timer}
    */
-  public Timer timer(String registry, String metricName, String... metricPath) {
-    return registry(registry).timer(mkName(metricName, metricPath));
+  public Timer timer(SolrInfoBean info, String registry, String metricName, String... metricPath) {
+    final String name = mkName(metricName, metricPath);
+    if (info != null) {
+      info.registerMetricName(name);
+    }
+    return registry(registry).timer(name);
   }
 
   /**
@@ -483,8 +566,12 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Counter}
    */
-  public Counter counter(String registry, String metricName, String... metricPath) {
-    return registry(registry).counter(mkName(metricName, metricPath));
+  public Counter counter(SolrInfoBean info, String registry, String metricName, String... metricPath) {
+    final String name = mkName(metricName, metricPath);
+    if (info != null) {
+      info.registerMetricName(name);
+    }
+    return registry(registry).counter(name);
   }
 
   /**
@@ -495,8 +582,12 @@ public class SolrMetricManager {
    * @param metricPath (optional) additional top-most metric name path elements
    * @return existing or a newly created {@link Histogram}
    */
-  public Histogram histogram(String registry, String metricName, String... metricPath) {
-    return registry(registry).histogram(mkName(metricName, metricPath));
+  public Histogram histogram(SolrInfoBean info, String registry, String metricName, String... metricPath) {
+    final String name = mkName(metricName, metricPath);
+    if (info != null) {
+      info.registerMetricName(name);
+    }
+    return registry(registry).histogram(name);
   }
 
   /**
@@ -510,9 +601,12 @@ public class SolrMetricManager {
    *                   using dotted notation
    * @param metricPath (optional) additional top-most metric name path elements
    */
-  public void register(String registry, Metric metric, boolean force, String metricName, String... metricPath) {
+  public void register(SolrInfoBean info, String registry, Metric metric, boolean force, String metricName, String... metricPath) {
     MetricRegistry metricRegistry = registry(registry);
     String fullName = mkName(metricName, metricPath);
+    if (info != null) {
+      info.registerMetricName(fullName);
+    }
     synchronized (metricRegistry) {
       if (force && metricRegistry.getMetrics().containsKey(fullName)) {
         metricRegistry.remove(fullName);
@@ -521,8 +615,8 @@ public class SolrMetricManager {
     }
   }
 
-  public void registerGauge(String registry, Gauge<?> gauge, boolean force, String metricName, String... metricPath) {
-    register(registry, gauge, force, metricName, metricPath);
+  public void registerGauge(SolrInfoBean info, String registry, Gauge<?> gauge, boolean force, String metricName, String... metricPath) {
+    register(info, registry, gauge, force, metricName, metricPath);
   }
 
   /**
@@ -569,7 +663,7 @@ public class SolrMetricManager {
    * </pre>
    * <b>NOTE:</b> Once a registry is renamed in a way that its metrics are combined with another repository
    * it is no longer possible to retrieve the original metrics until this renaming is removed and the Solr
-   * {@link org.apache.solr.core.SolrInfoMBean.Group} of components that reported to that name is restarted.
+   * {@link org.apache.solr.core.SolrInfoBean.Group} of components that reported to that name is restarted.
    * @param registry The name of the registry
    * @return A potentially overridden (via System properties) registry name
    */
@@ -600,7 +694,7 @@ public class SolrMetricManager {
    *              and the group parameter will be ignored.
    * @return fully-qualified and prefixed registry name, with overrides applied.
    */
-  public static String getRegistryName(SolrInfoMBean.Group group, String... names) {
+  public static String getRegistryName(SolrInfoBean.Group group, String... names) {
     String fullName;
     String prefix = REGISTRY_NAME_PREFIX + group.toString() + ".";
     // check for existing prefix and group
@@ -622,7 +716,7 @@ public class SolrMetricManager {
   // reporter management
 
   /**
-   * Create and register {@link SolrMetricReporter}-s specific to a {@link org.apache.solr.core.SolrInfoMBean.Group}.
+   * Create and register {@link SolrMetricReporter}-s specific to a {@link org.apache.solr.core.SolrInfoBean.Group}.
    * Note: reporters that specify neither "group" nor "registry" attributes are treated as universal -
    * they will always be loaded for any group. These two attributes may also contain multiple comma- or
    * whitespace-separated values, in which case the reporter will be loaded for any matching value from
@@ -634,7 +728,7 @@ public class SolrMetricManager {
    * @param group selected group, not null
    * @param registryNames optional child registry name elements
    */
-  public void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, String tag, SolrInfoMBean.Group group, String... registryNames) {
+  public void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, String tag, SolrInfoBean.Group group, String... registryNames) {
     if (pluginInfos == null || pluginInfos.length == 0) {
       return;
     }
@@ -941,13 +1035,13 @@ public class SolrMetricManager {
     // prepare default plugin if none present in the config
     Map<String, String> attrs = new HashMap<>();
     attrs.put("name", "shardDefault");
-    attrs.put("group", SolrInfoMBean.Group.shard.toString());
+    attrs.put("group", SolrInfoBean.Group.shard.toString());
     Map<String, Object> initArgs = new HashMap<>();
     initArgs.put("period", DEFAULT_CLOUD_REPORTER_PERIOD);
 
     String registryName = core.getCoreMetricManager().getRegistryName();
     // collect infos and normalize
-    List<PluginInfo> infos = prepareCloudPlugins(pluginInfos, SolrInfoMBean.Group.shard.toString(), SolrShardReporter.class.getName(),
+    List<PluginInfo> infos = prepareCloudPlugins(pluginInfos, SolrInfoBean.Group.shard.toString(), SolrShardReporter.class.getName(),
         attrs, initArgs, null);
     for (PluginInfo info : infos) {
       try {
@@ -967,12 +1061,12 @@ public class SolrMetricManager {
     }
     Map<String, String> attrs = new HashMap<>();
     attrs.put("name", "clusterDefault");
-    attrs.put("group", SolrInfoMBean.Group.cluster.toString());
+    attrs.put("group", SolrInfoBean.Group.cluster.toString());
     Map<String, Object> initArgs = new HashMap<>();
     initArgs.put("period", DEFAULT_CLOUD_REPORTER_PERIOD);
-    List<PluginInfo> infos = prepareCloudPlugins(pluginInfos, SolrInfoMBean.Group.cluster.toString(), SolrClusterReporter.class.getName(),
+    List<PluginInfo> infos = prepareCloudPlugins(pluginInfos, SolrInfoBean.Group.cluster.toString(), SolrClusterReporter.class.getName(),
         attrs, initArgs, null);
-    String registryName = getRegistryName(SolrInfoMBean.Group.cluster);
+    String registryName = getRegistryName(SolrInfoBean.Group.cluster);
     for (PluginInfo info : infos) {
       try {
         SolrMetricReporter reporter = loadReporter(registryName, cc.getResourceLoader(), info, null);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
index ff2d3fc..9ad15d0 100644
--- a/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
@@ -30,6 +30,7 @@ public abstract class SolrMetricReporter implements Closeable, PluginInfoInitial
   protected final String registryName;
   protected final SolrMetricManager metricManager;
   protected PluginInfo pluginInfo;
+  protected boolean enabled = true;
 
   /**
    * Create a reporter for metrics managed in a named registry.
@@ -58,6 +59,17 @@ public abstract class SolrMetricReporter implements Closeable, PluginInfoInitial
   }
 
   /**
+   * Enable reporting, defaults to true. Implementations should check this flag in
+   * {@link #validate()} and accordingly enable or disable reporting.
+   * @param enabled enable, defaults to true when null or not set.
+   */
+  public void setEnabled(Boolean enabled) {
+    if (enabled != null) {
+      this.enabled = enabled;
+    }
+  }
+
+  /**
    * Get the effective {@link PluginInfo} instance that was used for
    * initialization of this plugin.
    * @return plugin info, or null if not yet initialized.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java b/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
index 1f5b4f0..4298c18 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/JmxObjectNameFactory.java
@@ -51,6 +51,20 @@ public class JmxObjectNameFactory implements ObjectNameFactory {
   }
 
   /**
+   * Return current domain.
+   */
+  public String getDomain() {
+    return domain;
+  }
+
+  /**
+   * Return current reporterName.
+   */
+  public String getReporterName() {
+    return reporterName;
+  }
+
+  /**
    * Create a hierarchical name.
    *
    * @param type    metric class, eg. "counters", may be null for non-metric MBeans
@@ -60,7 +74,8 @@ public class JmxObjectNameFactory implements ObjectNameFactory {
   @Override
   public ObjectName createName(String type, String currentDomain, String name) {
     SolrMetricInfo metricInfo = SolrMetricInfo.of(name);
-
+    String safeName = metricInfo != null ? metricInfo.name : name;
+    safeName = safeName.replaceAll(":", "_");
     // It turns out that ObjectName(String) mostly preserves key ordering
     // as specified in the constructor (except for the 'type' key that ends
     // up at top level) - unlike ObjectName(String, Map) constructor
@@ -90,24 +105,42 @@ public class JmxObjectNameFactory implements ObjectNameFactory {
       sb.append(currentDomain);
       sb.append(':');
     }
-    sb.append("reporter=");
-    sb.append(reporterName);
-    sb.append(',');
+    if (props != null && props.length > 0) {
+      boolean added = false;
+      for (int i = 0; i < props.length; i += 2) {
+        if (props[i] == null || props[i].isEmpty()) {
+          continue;
+        }
+        if (props[i + 1] == null || props[i + 1].isEmpty()) {
+          continue;
+        }
+        sb.append(',');
+        sb.append(props[i]);
+        sb.append('=');
+        sb.append(props[i + 1]);
+        added = true;
+      }
+      if (added) {
+        sb.append(',');
+      }
+    }
     if (metricInfo != null) {
       sb.append("category=");
       sb.append(metricInfo.category.toString());
-      sb.append(",scope=");
-      sb.append(metricInfo.scope);
+      if (metricInfo.scope != null) {
+        sb.append(",scope=");
+        sb.append(metricInfo.scope);
+      }
       // we could also split by type, but don't call it 'type' :)
       // if (type != null) {
       //   sb.append(",class=");
       //   sb.append(type);
       // }
       sb.append(",name=");
-      sb.append(metricInfo.name);
+      sb.append(safeName);
     } else {
       // make dotted names into hierarchies
-      String[] path = name.split("\\.");
+      String[] path = safeName.split("\\.");
       for (int i = 0; i < path.length - 1; i++) {
         if (i > 0) {
           sb.append(',');
@@ -127,20 +160,6 @@ public class JmxObjectNameFactory implements ObjectNameFactory {
       sb.append("name=");
       sb.append(path[path.length - 1]);
     }
-    if (props != null && props.length > 0) {
-      for (int i = 0; i < props.length; i += 2) {
-        if (props[i] == null || props[i].isEmpty()) {
-          continue;
-        }
-        if (props[i + 1] == null || props[i + 1].isEmpty()) {
-          continue;
-        }
-        sb.append(',');
-        sb.append(props[i]);
-        sb.append('=');
-        sb.append(props[i + 1]);
-      }
-    }
 
     ObjectName objectName;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/ReporterClientCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/ReporterClientCache.java b/solr/core/src/java/org/apache/solr/metrics/reporters/ReporterClientCache.java
new file mode 100644
index 0000000..5745dec
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/ReporterClientCache.java
@@ -0,0 +1,84 @@
+/*
+ * 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.solr.metrics.reporters;
+
+import java.io.Closeable;
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple cache for reusable service clients used by some implementations of
+ * {@link org.apache.solr.metrics.SolrMetricReporter}.
+ */
+public class ReporterClientCache<T> implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, T> cache = new ConcurrentHashMap<>();
+
+  /**
+   * Provide an instance of service client.
+   * @param <T> formal type
+   */
+  public interface ClientProvider<T> {
+    /**
+     * Get an instance of a service client. It's not specified that each time this
+     * method is invoked a new client instance should be returned.
+     * @return client instance
+     * @throws Exception when client creation encountered an error.
+     */
+    T get() throws Exception;
+  }
+
+  /**
+   * Get existing or register a new client.
+   * @param id client id
+   * @param clientProvider provider of new client instances
+   */
+  public synchronized T getOrCreate(String id, ClientProvider<T> clientProvider) {
+    T item = cache.get(id);
+    if (item == null) {
+      try {
+        item = clientProvider.get();
+        cache.put(id, item);
+      } catch (Exception e) {
+        LOG.warn("Error providing a new client for id=" + id, e);
+        item = null;
+      }
+    }
+    return item;
+  }
+
+  /**
+   * Empty this cache, and close all clients that are {@link Closeable}.
+   */
+  public void close() {
+    for (T client : cache.values()) {
+      if (client instanceof Closeable) {
+        try {
+          ((Closeable)client).close();
+        } catch (Exception e) {
+          LOG.warn("Error closing client " + client + ", ignoring...", e);
+        }
+      }
+    }
+    cache.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGangliaReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGangliaReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGangliaReporter.java
index 45561e5..142ddd8 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGangliaReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGangliaReporter.java
@@ -17,6 +17,9 @@
 package org.apache.solr.metrics.reporters;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.MetricFilter;
@@ -24,21 +27,26 @@ import com.codahale.metrics.ganglia.GangliaReporter;
 import info.ganglia.gmetric4j.gmetric.GMetric;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *
  */
 public class SolrGangliaReporter extends SolrMetricReporter {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private String host = null;
   private int port = -1;
   private boolean multicast;
   private int period = 60;
   private String instancePrefix = null;
-  private String filterPrefix = null;
+  private List<String> filters = new ArrayList<>();
   private boolean testing;
   private GangliaReporter reporter;
 
+  private static final ReporterClientCache<GMetric> serviceRegistry = new ReporterClientCache<>();
+
   // for unit tests
   GMetric ganglia = null;
 
@@ -65,10 +73,24 @@ public class SolrGangliaReporter extends SolrMetricReporter {
     this.instancePrefix = prefix;
   }
 
-  public void setFilter(String filter) {
-    this.filterPrefix = filter;
+  /**
+   * Report only metrics with names matching any of the prefix filters.
+   * @param filters list of 0 or more prefixes. If the list is empty then
+   *                all names will match.
+   */
+  public void setFilter(List<String> filters) {
+    if (filters == null || filters.isEmpty()) {
+      return;
+    }
+    this.filters.addAll(filters);
   }
 
+  // due to vagaries of SolrPluginUtils.invokeSetters we need this too
+  public void setFilter(String filter) {
+    if (filter != null && !filter.isEmpty()) {
+      this.filters.add(filter);
+    }
+  }
 
   public void setPeriod(int period) {
     this.period = period;
@@ -89,6 +111,10 @@ public class SolrGangliaReporter extends SolrMetricReporter {
 
   @Override
   protected void validate() throws IllegalStateException {
+    if (!enabled) {
+      log.info("Reporter disabled for registry " + registryName);
+      return;
+    }
     if (host == null) {
       throw new IllegalStateException("Init argument 'host' must be set to a valid Ganglia server name.");
     }
@@ -106,12 +132,12 @@ public class SolrGangliaReporter extends SolrMetricReporter {
   //this is a separate method for unit tests
   void start() {
     if (!testing) {
-      try {
-        ganglia = new GMetric(host, port,
-            multicast ? GMetric.UDPAddressingMode.MULTICAST : GMetric.UDPAddressingMode.UNICAST,
-            1);
-      } catch (IOException ioe) {
-        throw new IllegalStateException("Exception connecting to Ganglia", ioe);
+      String id = host + ":" + port + ":" + multicast;
+      ganglia = serviceRegistry.getOrCreate(id, () -> new GMetric(host, port,
+          multicast ? GMetric.UDPAddressingMode.MULTICAST : GMetric.UDPAddressingMode.UNICAST,
+          1));
+      if (ganglia == null) {
+        return;
       }
     }
     if (instancePrefix == null) {
@@ -125,8 +151,8 @@ public class SolrGangliaReporter extends SolrMetricReporter {
         .convertDurationsTo(TimeUnit.MILLISECONDS)
         .prefixedWith(instancePrefix);
     MetricFilter filter;
-    if (filterPrefix != null) {
-      filter = new SolrMetricManager.PrefixFilter(filterPrefix);
+    if (!filters.isEmpty()) {
+      filter = new SolrMetricManager.PrefixFilter(filters);
     } else {
       filter = MetricFilter.ALL;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGraphiteReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGraphiteReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGraphiteReporter.java
index 8565ce8..d5b7a20 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGraphiteReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrGraphiteReporter.java
@@ -18,6 +18,8 @@ package org.apache.solr.metrics.reporters;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.MetricFilter;
@@ -41,9 +43,11 @@ public class SolrGraphiteReporter extends SolrMetricReporter {
   private int period = 60;
   private boolean pickled = false;
   private String instancePrefix = null;
-  private String filterPrefix = null;
+  private List<String> filters = new ArrayList<>();
   private GraphiteReporter reporter = null;
 
+  private static final ReporterClientCache<GraphiteSender> serviceRegistry = new ReporterClientCache<>();
+
   /**
    * Create a Graphite reporter for metrics managed in a named registry.
    *
@@ -67,10 +71,25 @@ public class SolrGraphiteReporter extends SolrMetricReporter {
     this.instancePrefix = prefix;
   }
 
+  /**
+   * Report only metrics with names matching any of the prefix filters.
+   * @param filters list of 0 or more prefixes. If the list is empty then
+   *                all names will match.
+   */
+  public void setFilter(List<String> filters) {
+    if (filters == null || filters.isEmpty()) {
+      return;
+    }
+    this.filters.addAll(filters);
+  }
+
   public void setFilter(String filter) {
-    this.filterPrefix = filter;
+    if (filter != null && !filter.isEmpty()) {
+      this.filters.add(filter);
+    }
   }
 
+
   public void setPickled(boolean pickled) {
     this.pickled = pickled;
   }
@@ -81,6 +100,10 @@ public class SolrGraphiteReporter extends SolrMetricReporter {
 
   @Override
   protected void validate() throws IllegalStateException {
+    if (!enabled) {
+      log.info("Reporter disabled for registry " + registryName);
+      return;
+    }
     if (host == null) {
       throw new IllegalStateException("Init argument 'host' must be set to a valid Graphite server name.");
     }
@@ -93,12 +116,15 @@ public class SolrGraphiteReporter extends SolrMetricReporter {
     if (period < 1) {
       throw new IllegalStateException("Init argument 'period' is in time unit 'seconds' and must be at least 1.");
     }
-    final GraphiteSender graphite;
-    if (pickled) {
-      graphite = new PickledGraphite(host, port);
-    } else {
-      graphite = new Graphite(host, port);
-    }
+    GraphiteSender graphite;
+    String id = host + ":" + port + ":" + pickled;
+    graphite = serviceRegistry.getOrCreate(id, () -> {
+      if (pickled) {
+        return new PickledGraphite(host, port);
+      } else {
+        return new Graphite(host, port);
+      }
+    });
     if (instancePrefix == null) {
       instancePrefix = registryName;
     } else {
@@ -110,8 +136,8 @@ public class SolrGraphiteReporter extends SolrMetricReporter {
         .convertRatesTo(TimeUnit.SECONDS)
         .convertDurationsTo(TimeUnit.MILLISECONDS);
     MetricFilter filter;
-    if (filterPrefix != null) {
-      filter = new SolrMetricManager.PrefixFilter(filterPrefix);
+    if (!filters.isEmpty()) {
+      filter = new SolrMetricManager.PrefixFilter(filters);
     } else {
       filter = MetricFilter.ALL;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
index 0e78eee..d09e043 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
@@ -16,15 +16,25 @@
  */
 package org.apache.solr.metrics.reporters;
 
+import javax.management.InstanceNotFoundException;
 import javax.management.MBeanServer;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
 
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Locale;
+import java.util.Set;
 
+import com.codahale.metrics.Gauge;
 import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricRegistryListener;
 import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
 import org.apache.solr.util.JmxUtil;
@@ -34,17 +44,25 @@ import org.slf4j.LoggerFactory;
 /**
  * A {@link SolrMetricReporter} that finds (or creates) a MBeanServer from
  * the given configuration and registers metrics to it with JMX.
+ * <p>NOTE: {@link JmxReporter} that this class uses exports only newly added metrics (it doesn't
+ * process already existing metrics in a registry)</p>
  */
 public class SolrJmxReporter extends SolrMetricReporter {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+  private static final ReporterClientCache<MBeanServer> serviceRegistry = new ReporterClientCache<>();
+
   private String domain;
   private String agentId;
   private String serviceUrl;
+  private String rootName;
+  private List<String> filters = new ArrayList<>();
 
   private JmxReporter reporter;
+  private MetricRegistry registry;
   private MBeanServer mBeanServer;
+  private MetricsMapListener listener;
 
   /**
    * Creates a new instance of {@link SolrJmxReporter}.
@@ -57,7 +75,7 @@ public class SolrJmxReporter extends SolrMetricReporter {
   }
 
   /**
-   * Initializes the reporter by finding (or creating) a MBeanServer
+   * Initializes the reporter by finding an MBeanServer
    * and registering the metricManager's metric registry.
    *
    * @param pluginInfo the configuration for the reporter
@@ -65,44 +83,62 @@ public class SolrJmxReporter extends SolrMetricReporter {
   @Override
   public synchronized void init(PluginInfo pluginInfo) {
     super.init(pluginInfo);
-
+    if (!enabled) {
+      log.info("Reporter disabled for registry " + registryName);
+      return;
+    }
+    log.debug("Initializing for registry " + registryName);
     if (serviceUrl != null && agentId != null) {
-      ManagementFactory.getPlatformMBeanServer(); // Ensure at least one MBeanServer is available.
       mBeanServer = JmxUtil.findFirstMBeanServer();
-      log.warn("No more than one of serviceUrl(%s) and agentId(%s) should be configured, using first MBeanServer instead of configuration.",
+      log.warn("No more than one of serviceUrl({}) and agentId({}) should be configured, using first MBeanServer instead of configuration.",
           serviceUrl, agentId, mBeanServer);
-    }
-    else if (serviceUrl != null) {
-      try {
-        mBeanServer = JmxUtil.findMBeanServerForServiceUrl(serviceUrl);
-      } catch (IOException e) {
-        log.warn("findMBeanServerForServiceUrl(%s) exception: %s", serviceUrl, e);
-        mBeanServer = null;
-      }
-    }
-    else if (agentId != null) {
+    } else if (serviceUrl != null) {
+      // reuse existing services
+      mBeanServer = serviceRegistry.getOrCreate(serviceUrl, () -> JmxUtil.findMBeanServerForServiceUrl(serviceUrl));
+    } else if (agentId != null) {
       mBeanServer = JmxUtil.findMBeanServerForAgentId(agentId);
     } else {
-      ManagementFactory.getPlatformMBeanServer(); // Ensure at least one MBeanServer is available.
       mBeanServer = JmxUtil.findFirstMBeanServer();
-      log.warn("No serviceUrl or agentId was configured, using first MBeanServer.", mBeanServer);
+      log.debug("No serviceUrl or agentId was configured, using first MBeanServer: " + mBeanServer);
     }
 
     if (mBeanServer == null) {
-      log.warn("No JMX server found. Not exposing Solr metrics.");
+      log.warn("No JMX server found. Not exposing Solr metrics via JMX.");
       return;
     }
 
-    JmxObjectNameFactory jmxObjectNameFactory = new JmxObjectNameFactory(pluginInfo.name, domain);
+    if (domain == null || domain.isEmpty()) {
+      domain = registryName;
+    }
+    String fullDomain = domain;
+    if (rootName != null && !rootName.isEmpty()) {
+      fullDomain = rootName + "." + domain;
+    }
+    JmxObjectNameFactory jmxObjectNameFactory = new JmxObjectNameFactory(pluginInfo.name, fullDomain);
+    registry = metricManager.registry(registryName);
+    // filter out MetricsMap gauges - we have a better way of handling them
+    MetricFilter mmFilter = (name, metric) -> !(metric instanceof MetricsMap);
+    MetricFilter filter;
+    if (filters.isEmpty()) {
+      filter = mmFilter;
+    } else {
+      // apply also prefix filters
+      SolrMetricManager.PrefixFilter prefixFilter = new SolrMetricManager.PrefixFilter(filters);
+      filter = new SolrMetricManager.AndFilter(prefixFilter, mmFilter);
+    }
 
-    reporter = JmxReporter.forRegistry(metricManager.registry(registryName))
+    reporter = JmxReporter.forRegistry(registry)
                           .registerWith(mBeanServer)
-                          .inDomain(domain)
+                          .inDomain(fullDomain)
+                          .filter(filter)
                           .createsObjectNamesWith(jmxObjectNameFactory)
                           .build();
     reporter.start();
+    // workaround for inability to register custom MBeans (to be available in metrics 4.0?)
+    listener = new MetricsMapListener(mBeanServer, jmxObjectNameFactory);
+    registry.addListener(listener);
 
-    log.info("JMX monitoring enabled at server: " + mBeanServer);
+    log.info("JMX monitoring for '" + fullDomain + "' (registry '" + registryName + "') enabled at server: " + mBeanServer);
   }
 
   /**
@@ -114,6 +150,11 @@ public class SolrJmxReporter extends SolrMetricReporter {
       reporter.close();
       reporter = null;
     }
+    if (listener != null && registry != null) {
+      registry.removeListener(listener);
+      listener.close();
+      listener = null;
+    }
   }
 
   /**
@@ -127,9 +168,19 @@ public class SolrJmxReporter extends SolrMetricReporter {
     // Nothing to validate
   }
 
+
+  /**
+   * Set root name of the JMX hierarchy for this reporter. Default (null or empty) is none, ie.
+   * the hierarchy will start from the domain name.
+   * @param rootName root name of the JMX name hierarchy, or null or empty for default.
+   */
+  public void setRootName(String rootName) {
+    this.rootName = rootName;
+  }
+
   /**
    * Sets the domain with which MBeans are published. If none is set,
-   * the domain defaults to the name of the core.
+   * the domain defaults to the name of the registry.
    *
    * @param domain the domain
    */
@@ -162,7 +213,46 @@ public class SolrJmxReporter extends SolrMetricReporter {
   }
 
   /**
-   * Retrieves the reporter's MBeanServer.
+   * Return configured agentId or null.
+   */
+  public String getAgentId() {
+    return agentId;
+  }
+
+  /**
+   * Return configured serviceUrl or null.
+   */
+  public String getServiceUrl() {
+    return serviceUrl;
+  }
+
+  /**
+   * Return configured domain or null.
+   */
+  public String getDomain() {
+    return domain;
+  }
+
+  /**
+   * Report only metrics with names matching any of the prefix filters.
+   * @param filters list of 0 or more prefixes. If the list is empty then
+   *                all names will match.
+   */
+  public void setFilter(List<String> filters) {
+    if (filters == null || filters.isEmpty()) {
+      return;
+    }
+    this.filters.addAll(filters);
+  }
+
+  public void setFilter(String filter) {
+    if (filter != null && !filter.isEmpty()) {
+      this.filters.add(filter);
+    }
+  }
+
+  /**
+   * Return the reporter's MBeanServer.
    *
    * @return the reporter's MBeanServer
    */
@@ -170,10 +260,72 @@ public class SolrJmxReporter extends SolrMetricReporter {
     return mBeanServer;
   }
 
+  /**
+   * For unit tests.
+   * @return true if this reporter is actively reporting metrics to JMX.
+   */
+  public boolean isActive() {
+    return reporter != null;
+  }
+
   @Override
   public String toString() {
-    return String.format(Locale.ENGLISH, "[%s@%s: domain = %s, service url = %s, agent id = %s]",
-        getClass().getName(), Integer.toHexString(hashCode()), domain, serviceUrl, agentId);
+    return String.format(Locale.ENGLISH, "[%s@%s: rootName = %s, domain = %s, service url = %s, agent id = %s]",
+        getClass().getName(), Integer.toHexString(hashCode()), rootName, domain, serviceUrl, agentId);
   }
 
+  private static class MetricsMapListener extends MetricRegistryListener.Base {
+    MBeanServer server;
+    JmxObjectNameFactory nameFactory;
+    // keep the names so that we can unregister them on core close
+    Set<ObjectName> registered = new HashSet<>();
+
+    MetricsMapListener(MBeanServer server, JmxObjectNameFactory nameFactory) {
+      this.server = server;
+      this.nameFactory = nameFactory;
+    }
+
+    @Override
+    public void onGaugeAdded(String name, Gauge<?> gauge) {
+      if (!(gauge instanceof MetricsMap)) {
+        return;
+      }
+      synchronized (server) {
+        try {
+          ObjectName objectName = nameFactory.createName("gauges", nameFactory.getDomain(), name);
+          log.debug("REGISTER " + objectName);
+          if (registered.contains(objectName) || server.isRegistered(objectName)) {
+            log.debug("-unregistering old instance of " + objectName);
+            try {
+              server.unregisterMBean(objectName);
+            } catch (InstanceNotFoundException e) {
+              // ignore
+            }
+          }
+          // some MBean servers re-write object name to include additional properties
+          ObjectInstance instance = server.registerMBean(gauge, objectName);
+          if (instance != null) {
+            registered.add(instance.getObjectName());
+          }
+        } catch (Exception e) {
+          log.warn("bean registration error", e);
+        }
+      }
+    }
+
+    public void close() {
+      synchronized (server) {
+        for (ObjectName name : registered) {
+          try {
+            if (server.isRegistered(name)) {
+              server.unregisterMBean(name);
+            }
+          } catch (Exception e) {
+            log.debug("bean unregistration error", e);
+          }
+        }
+        registered.clear();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/SolrSlf4jReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrSlf4jReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrSlf4jReporter.java
index 817dda1..8b7c35e 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/SolrSlf4jReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrSlf4jReporter.java
@@ -18,6 +18,8 @@ package org.apache.solr.metrics.reporters;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.MetricFilter;
@@ -47,7 +49,7 @@ public class SolrSlf4jReporter extends SolrMetricReporter {
   private int period = 60;
   private String instancePrefix = null;
   private String logger = null;
-  private String filterPrefix = null;
+  private List<String> filters = new ArrayList<>();
   private Slf4jReporter reporter;
 
   /**
@@ -65,10 +67,25 @@ public class SolrSlf4jReporter extends SolrMetricReporter {
     this.instancePrefix = prefix;
   }
 
+  /**
+   * Report only metrics with names matching any of the prefix filters.
+   * @param filters list of 0 or more prefixes. If the list is empty then
+   *                all names will match.
+   */
+  public void setFilter(List<String> filters) {
+    if (filters == null || filters.isEmpty()) {
+      return;
+    }
+    this.filters.addAll(filters);
+  }
+
   public void setFilter(String filter) {
-    this.filterPrefix = filter;
+    if (filter != null && !filter.isEmpty()) {
+      this.filters.add(filter);
+    }
   }
 
+
   public void setLogger(String logger) {
     this.logger = logger;
   }
@@ -79,6 +96,10 @@ public class SolrSlf4jReporter extends SolrMetricReporter {
 
   @Override
   protected void validate() throws IllegalStateException {
+    if (!enabled) {
+      log.info("Reporter disabled for registry " + registryName);
+      return;
+    }
     if (period < 1) {
       throw new IllegalStateException("Init argument 'period' is in time unit 'seconds' and must be at least 1.");
     }
@@ -93,8 +114,8 @@ public class SolrSlf4jReporter extends SolrMetricReporter {
         .convertDurationsTo(TimeUnit.MILLISECONDS);
 
     MetricFilter filter;
-    if (filterPrefix != null) {
-      filter = new SolrMetricManager.PrefixFilter(filterPrefix);
+    if (!filters.isEmpty()) {
+      filter = new SolrMetricManager.PrefixFilter(filters);
     } else {
       filter = MetricFilter.ALL;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
index a34accd..c437457 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrClusterReporter.java
@@ -33,7 +33,7 @@ import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.handler.admin.MetricsCollectorHandler;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricReporter;
@@ -92,14 +92,14 @@ import static org.apache.solr.common.params.CommonParams.ID;
 public class SolrClusterReporter extends SolrMetricReporter {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String CLUSTER_GROUP = SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.cluster.toString());
+  public static final String CLUSTER_GROUP = SolrMetricManager.overridableRegistryName(SolrInfoBean.Group.cluster.toString());
 
   public static final List<SolrReporter.Report> DEFAULT_REPORTS = new ArrayList<SolrReporter.Report>() {{
     add(new SolrReporter.Report(CLUSTER_GROUP, "jetty",
-        SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.jetty.toString()),
+        SolrMetricManager.overridableRegistryName(SolrInfoBean.Group.jetty.toString()),
         Collections.emptySet())); // all metrics
     add(new SolrReporter.Report(CLUSTER_GROUP, "jvm",
-        SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.jvm.toString()),
+        SolrMetricManager.overridableRegistryName(SolrInfoBean.Group.jvm.toString()),
         new HashSet<String>() {{
           add("memory\\.total\\..*");
           add("memory\\.heap\\..*");
@@ -109,7 +109,7 @@ public class SolrClusterReporter extends SolrMetricReporter {
           add("os\\.OpenFileDescriptorCount");
           add("threads\\.count");
         }}));
-    add(new SolrReporter.Report(CLUSTER_GROUP, "node", SolrMetricManager.overridableRegistryName(SolrInfoMBean.Group.node.toString()),
+    add(new SolrReporter.Report(CLUSTER_GROUP, "node", SolrMetricManager.overridableRegistryName(SolrInfoBean.Group.node.toString()),
         new HashSet<String>() {{
           add("CONTAINER\\.cores\\..*");
           add("CONTAINER\\.fs\\..*");
@@ -159,6 +159,16 @@ public class SolrClusterReporter extends SolrMetricReporter {
     });
   }
 
+  public void setReport(Map map) {
+    if (map == null || map.isEmpty()) {
+      return;
+    }
+    SolrReporter.Report r = SolrReporter.Report.fromMap(map);
+    if (r != null) {
+      reports.add(r);
+    }
+  }
+
   // for unit tests
   int getPeriod() {
     return period;
@@ -170,9 +180,6 @@ public class SolrClusterReporter extends SolrMetricReporter {
 
   @Override
   protected void validate() throws IllegalStateException {
-    if (period < 1) {
-      log.info("Turning off node reporter, period=" + period);
-    }
     if (reports.isEmpty()) { // set defaults
       reports = DEFAULT_REPORTS;
     }
@@ -189,12 +196,17 @@ public class SolrClusterReporter extends SolrMetricReporter {
     if (reporter != null) {
       reporter.close();;
     }
+    if (!enabled) {
+      log.info("Reporter disabled for registry " + registryName);
+      return;
+    }
     // start reporter only in cloud mode
     if (!cc.isZooKeeperAware()) {
       log.warn("Not ZK-aware, not starting...");
       return;
     }
     if (period < 1) { // don't start it
+      log.info("Turning off node reporter, period=" + period);
       return;
     }
     HttpClient httpClient = cc.getUpdateShardHandler().getHttpClient();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
index 8b36d3e..b36c596 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
@@ -98,7 +98,13 @@ public class SolrShardReporter extends SolrMetricReporter {
     if (filterConfig == null || filterConfig.isEmpty()) {
       return;
     }
-    filters = filterConfig;
+    filters.addAll(filterConfig);
+  }
+
+  public void setFilter(String filter) {
+    if (filter != null && !filter.isEmpty()) {
+      this.filters.add(filter);
+    }
   }
 
   // for unit tests
@@ -108,9 +114,6 @@ public class SolrShardReporter extends SolrMetricReporter {
 
   @Override
   protected void validate() throws IllegalStateException {
-    if (period < 1) {
-      log.info("Turning off shard reporter, period=" + period);
-    }
     if (filters.isEmpty()) {
       filters = DEFAULT_FILTERS;
     }
@@ -128,13 +131,17 @@ public class SolrShardReporter extends SolrMetricReporter {
     if (reporter != null) {
       reporter.close();
     }
+    if (!enabled) {
+      log.info("Reporter disabled for registry " + registryName);
+      return;
+    }
     if (core.getCoreDescriptor().getCloudDescriptor() == null) {
       // not a cloud core
       log.warn("Not initializing shard reporter for non-cloud core " + core.getName());
       return;
     }
     if (period < 1) { // don't start it
-      log.warn("Not starting shard reporter ");
+      log.warn("period=" + period + ", not starting shard reporter ");
       return;
     }
     // our id is coreNodeName

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/request/SolrRequestHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestHandler.java b/solr/core/src/java/org/apache/solr/request/SolrRequestHandler.java
index 82ce2e0..8350f9e 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestHandler.java
@@ -17,7 +17,7 @@
 package org.apache.solr.request;
 
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.response.SolrQueryResponse;
 
 /**
@@ -38,7 +38,7 @@ import org.apache.solr.response.SolrQueryResponse;
  *
  *
  */
-public interface SolrRequestHandler extends SolrInfoMBean {
+public interface SolrRequestHandler extends SolrInfoBean {
 
   /** <code>init</code> will be called just once, immediately after creation.
    * <p>The args are user-level initialization parameters that

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
----------------------------------------------------------------------
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 9c4b892..cb699b2 100644
--- a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
@@ -15,15 +15,17 @@
  * limitations under the License.
  */
 package org.apache.solr.search;
+
+import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.ConcurrentLRUCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SimpleOrderedMap;
 
-import java.io.Serializable;
 import java.lang.invoke.MethodHandles;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -57,6 +59,10 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>
 
   private long maxRamBytes;
 
+  private MetricsMap cacheMap;
+  private Set<String> metricNames = new HashSet<>();
+  private MetricRegistry registry;
+
   @Override
   public Object init(Map args, Object persistence, CacheRegenerator regenerator) {
     super.init(args, regenerator);
@@ -215,68 +221,80 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>
   }
 
   @Override
-  public String getSource() {
-    return null;
+  public Set<String> getMetricNames() {
+    return metricNames;
   }
 
-
   @Override
-  public NamedList getStatistics() {
-    NamedList<Serializable> lst = new SimpleOrderedMap<>();
-    if (cache == null)  return lst;
-    ConcurrentLRUCache.Stats stats = cache.getStats();
-    long lookups = stats.getCumulativeLookups();
-    long hits = stats.getCumulativeHits();
-    long inserts = stats.getCumulativePuts();
-    long evictions = stats.getCumulativeEvictions();
-    long size = stats.getCurrentSize();
-    long clookups = 0;
-    long chits = 0;
-    long cinserts = 0;
-    long cevictions = 0;
-
-    // NOTE: It is safe to iterate on a CopyOnWriteArrayList
-    for (ConcurrentLRUCache.Stats statistiscs : statsList) {
-      clookups += statistiscs.getCumulativeLookups();
-      chits += statistiscs.getCumulativeHits();
-      cinserts += statistiscs.getCumulativePuts();
-      cevictions += statistiscs.getCumulativeEvictions();
-    }
+  public void initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    registry = manager.registry(registryName);
+    cacheMap = new MetricsMap((detailed, map) -> {
+      if (cache != null) {
+        ConcurrentLRUCache.Stats stats = cache.getStats();
+        long lookups = stats.getCumulativeLookups();
+        long hits = stats.getCumulativeHits();
+        long inserts = stats.getCumulativePuts();
+        long evictions = stats.getCumulativeEvictions();
+        long size = stats.getCurrentSize();
+        long clookups = 0;
+        long chits = 0;
+        long cinserts = 0;
+        long cevictions = 0;
+
+        // NOTE: It is safe to iterate on a CopyOnWriteArrayList
+        for (ConcurrentLRUCache.Stats statistiscs : statsList) {
+          clookups += statistiscs.getCumulativeLookups();
+          chits += statistiscs.getCumulativeHits();
+          cinserts += statistiscs.getCumulativePuts();
+          cevictions += statistiscs.getCumulativeEvictions();
+        }
+
+        map.put("lookups", lookups);
+        map.put("hits", hits);
+        map.put("hitratio", calcHitRatio(lookups, hits));
+        map.put("inserts", inserts);
+        map.put("evictions", evictions);
+        map.put("size", size);
+
+        map.put("warmupTime", warmupTime);
+        map.put("cumulative_lookups", clookups);
+        map.put("cumulative_hits", chits);
+        map.put("cumulative_hitratio", calcHitRatio(clookups, chits));
+        map.put("cumulative_inserts", cinserts);
+        map.put("cumulative_evictions", cevictions);
+
+        if (detailed && showItems != 0) {
+          Map items = cache.getLatestAccessedItems( showItems == -1 ? Integer.MAX_VALUE : showItems );
+          for (Map.Entry e : (Set <Map.Entry>)items.entrySet()) {
+            Object k = e.getKey();
+            Object v = e.getValue();
+
+            String ks = "item_" + k;
+            String vs = v.toString();
+            map.put(ks,vs);
+          }
 
-    lst.add("lookups", lookups);
-    lst.add("hits", hits);
-    lst.add("hitratio", calcHitRatio(lookups, hits));
-    lst.add("inserts", inserts);
-    lst.add("evictions", evictions);
-    lst.add("size", size);
-
-    lst.add("warmupTime", warmupTime);
-    lst.add("cumulative_lookups", clookups);
-    lst.add("cumulative_hits", chits);
-    lst.add("cumulative_hitratio", calcHitRatio(clookups, chits));
-    lst.add("cumulative_inserts", cinserts);
-    lst.add("cumulative_evictions", cevictions);
-
-    if (showItems != 0) {
-      Map items = cache.getLatestAccessedItems( showItems == -1 ? Integer.MAX_VALUE : showItems );
-      for (Map.Entry e : (Set <Map.Entry>)items.entrySet()) {
-        Object k = e.getKey();
-        Object v = e.getValue();
-
-        String ks = "item_" + k;
-        String vs = v.toString();
-        lst.add(ks,vs);
+        }
       }
-      
-    }
+    });
+    manager.registerGauge(this, registryName, cacheMap, true, scope, getCategory().toString());
+  }
+
+  // for unit tests only
+  MetricsMap getMetricsMap() {
+    return cacheMap;
+  }
 
-    return lst;
+  @Override
+  public MetricRegistry getMetricRegistry() {
+    return registry;
   }
 
   @Override
   public String toString() {
-    return name() + getStatistics().toString();
+    return name() + cacheMap != null ? cacheMap.getValue().toString() : "";
   }
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/search/LFUCache.java
----------------------------------------------------------------------
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 2b593c6..82ba6d2 100644
--- a/solr/core/src/java/org/apache/solr/search/LFUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LFUCache.java
@@ -15,19 +15,19 @@
  * limitations under the License.
  */
 package org.apache.solr.search;
-import java.io.Serializable;
+
 import java.lang.invoke.MethodHandles;
-import java.net.URL;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.SolrCore;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.ConcurrentLFUCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -64,6 +64,9 @@ public class LFUCache<K, V> implements SolrCache<K, V> {
   private ConcurrentLFUCache<K, V> cache;
   private int showItems = 0;
   private Boolean timeDecay = true;
+  private MetricsMap cacheMap;
+  private Set<String> metricNames = new HashSet<>();
+  private MetricRegistry registry;
 
   @Override
   public Object init(Map args, Object persistence, CacheRegenerator regenerator) {
@@ -212,11 +215,6 @@ public class LFUCache<K, V> implements SolrCache<K, V> {
   }
 
   @Override
-  public String getVersion() {
-    return SolrCore.version;
-  }
-
-  @Override
   public String getDescription() {
     return description;
   }
@@ -226,16 +224,6 @@ public class LFUCache<K, V> implements SolrCache<K, V> {
     return Category.CACHE;
   }
 
-  @Override
-  public String getSource() {
-    return null;
-  }
-
-  @Override
-  public URL[] getDocs() {
-    return null;
-  }
-
   // returns a ratio, not a percent.
   private static String calcHitRatio(long lookups, long hits) {
     if (lookups == 0) return "0.00";
@@ -246,62 +234,81 @@ public class LFUCache<K, V> implements SolrCache<K, V> {
   }
 
   @Override
-  public NamedList getStatistics() {
-    NamedList<Serializable> lst = new SimpleOrderedMap<>();
-    if (cache == null) return lst;
-    ConcurrentLFUCache.Stats stats = cache.getStats();
-    long lookups = stats.getCumulativeLookups();
-    long hits = stats.getCumulativeHits();
-    long inserts = stats.getCumulativePuts();
-    long evictions = stats.getCumulativeEvictions();
-    long size = stats.getCurrentSize();
-
-    lst.add("lookups", lookups);
-    lst.add("hits", hits);
-    lst.add("hitratio", calcHitRatio(lookups, hits));
-    lst.add("inserts", inserts);
-    lst.add("evictions", evictions);
-    lst.add("size", size);
-
-    lst.add("warmupTime", warmupTime);
-    lst.add("timeDecay", timeDecay);
-
-    long clookups = 0;
-    long chits = 0;
-    long cinserts = 0;
-    long cevictions = 0;
-
-    // NOTE: It is safe to iterate on a CopyOnWriteArrayList
-    for (ConcurrentLFUCache.Stats statistics : statsList) {
-      clookups += statistics.getCumulativeLookups();
-      chits += statistics.getCumulativeHits();
-      cinserts += statistics.getCumulativePuts();
-      cevictions += statistics.getCumulativeEvictions();
-    }
-    lst.add("cumulative_lookups", clookups);
-    lst.add("cumulative_hits", chits);
-    lst.add("cumulative_hitratio", calcHitRatio(clookups, chits));
-    lst.add("cumulative_inserts", cinserts);
-    lst.add("cumulative_evictions", cevictions);
-
-    if (showItems != 0) {
-      Map items = cache.getMostUsedItems(showItems == -1 ? Integer.MAX_VALUE : showItems);
-      for (Map.Entry e : (Set<Map.Entry>) items.entrySet()) {
-        Object k = e.getKey();
-        Object v = e.getValue();
-
-        String ks = "item_" + k;
-        String vs = v.toString();
-        lst.add(ks, vs);
+  public void initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    registry = manager.registry(registryName);
+    cacheMap = new MetricsMap((detailed, map) -> {
+      if (cache != null) {
+        ConcurrentLFUCache.Stats stats = cache.getStats();
+        long lookups = stats.getCumulativeLookups();
+        long hits = stats.getCumulativeHits();
+        long inserts = stats.getCumulativePuts();
+        long evictions = stats.getCumulativeEvictions();
+        long size = stats.getCurrentSize();
+
+        map.put("lookups", lookups);
+        map.put("hits", hits);
+        map.put("hitratio", calcHitRatio(lookups, hits));
+        map.put("inserts", inserts);
+        map.put("evictions", evictions);
+        map.put("size", size);
+
+        map.put("warmupTime", warmupTime);
+        map.put("timeDecay", timeDecay);
+
+        long clookups = 0;
+        long chits = 0;
+        long cinserts = 0;
+        long cevictions = 0;
+
+        // NOTE: It is safe to iterate on a CopyOnWriteArrayList
+        for (ConcurrentLFUCache.Stats statistics : statsList) {
+          clookups += statistics.getCumulativeLookups();
+          chits += statistics.getCumulativeHits();
+          cinserts += statistics.getCumulativePuts();
+          cevictions += statistics.getCumulativeEvictions();
+        }
+        map.put("cumulative_lookups", clookups);
+        map.put("cumulative_hits", chits);
+        map.put("cumulative_hitratio", calcHitRatio(clookups, chits));
+        map.put("cumulative_inserts", cinserts);
+        map.put("cumulative_evictions", cevictions);
+
+        if (detailed && showItems != 0) {
+          Map items = cache.getMostUsedItems(showItems == -1 ? Integer.MAX_VALUE : showItems);
+          for (Map.Entry e : (Set<Map.Entry>) items.entrySet()) {
+            Object k = e.getKey();
+            Object v = e.getValue();
+
+            String ks = "item_" + k;
+            String vs = v.toString();
+            map.put(ks, vs);
+          }
+
+        }
+
       }
+    });
+    manager.registerGauge(this, registryName, cacheMap, true, scope, getCategory().toString());
+  }
 
-    }
+  // for unit tests only
+  MetricsMap getMetricsMap() {
+    return cacheMap;
+  }
 
-    return lst;
+  @Override
+  public Set<String> getMetricNames() {
+    return metricNames;
+  }
+
+  @Override
+  public MetricRegistry getMetricRegistry() {
+    return registry;
   }
 
   @Override
   public String toString() {
-    return name + getStatistics().toString();
+    return name + cacheMap != null ? cacheMap.getValue().toString() : "";
   }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30cc70f/solr/core/src/java/org/apache/solr/search/LRUCache.java
----------------------------------------------------------------------
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 b178fb2..ce206fe 100644
--- a/solr/core/src/java/org/apache/solr/search/LRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LRUCache.java
@@ -19,18 +19,21 @@ package org.apache.solr.search;
 import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.LongAdder;
 
+import com.codahale.metrics.MetricRegistry;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,6 +58,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   static final long LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY =
       HASHTABLE_RAM_BYTES_PER_ENTRY
           + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // previous & next references
+
   /// End copied code
 
   /* An instance of this class will be shared across multiple instances
@@ -82,6 +86,9 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
 
   private Map<K,V> map;
   private String description="LRU Cache";
+  private MetricsMap cacheMap;
+  private Set<String> metricNames = new HashSet<>();
+  private MetricRegistry registry;
 
   private long maxRamBytes = Long.MAX_VALUE;
   // The synchronization used for the map will be used to update this,
@@ -319,45 +326,56 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   }
 
   @Override
-  public String getSource() {
-    return null;
+  public Set<String> getMetricNames() {
+    return metricNames;
   }
 
   @Override
-  public NamedList getStatistics() {
-    NamedList lst = new SimpleOrderedMap();
-    synchronized (map) {
-      lst.add("lookups", lookups);
-      lst.add("hits", hits);
-      lst.add("hitratio", calcHitRatio(lookups,hits));
-      lst.add("inserts", inserts);
-      lst.add("evictions", evictions);
-      lst.add("size", map.size());
+  public void initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    registry = manager.registry(registryName);
+    cacheMap = new MetricsMap((detailed, res) -> {
+      synchronized (map) {
+        res.put("lookups", lookups);
+        res.put("hits", hits);
+        res.put("hitratio", calcHitRatio(lookups,hits));
+        res.put("inserts", inserts);
+        res.put("evictions", evictions);
+        res.put("size", map.size());
+        if (maxRamBytes != Long.MAX_VALUE)  {
+          res.put("maxRamMB", maxRamBytes / 1024L / 1024L);
+          res.put("ramBytesUsed", ramBytesUsed());
+          res.put("evictionsRamUsage", evictionsRamUsage);
+        }
+      }
+      res.put("warmupTime", warmupTime);
+
+      long clookups = stats.lookups.longValue();
+      long chits = stats.hits.longValue();
+      res.put("cumulative_lookups", clookups);
+      res.put("cumulative_hits", chits);
+      res.put("cumulative_hitratio", calcHitRatio(clookups, chits));
+      res.put("cumulative_inserts", stats.inserts.longValue());
+      res.put("cumulative_evictions", stats.evictions.longValue());
       if (maxRamBytes != Long.MAX_VALUE)  {
-        lst.add("maxRamMB", maxRamBytes / 1024L / 1024L);
-        lst.add("ramBytesUsed", ramBytesUsed());
-        lst.add("evictionsRamUsage", evictionsRamUsage);
+        res.put("cumulative_evictionsRamUsage", stats.evictionsRamUsage.longValue());
       }
-    }
-    lst.add("warmupTime", warmupTime);
-    
-    long clookups = stats.lookups.longValue();
-    long chits = stats.hits.longValue();
-    lst.add("cumulative_lookups", clookups);
-    lst.add("cumulative_hits", chits);
-    lst.add("cumulative_hitratio", calcHitRatio(clookups, chits));
-    lst.add("cumulative_inserts", stats.inserts.longValue());
-    lst.add("cumulative_evictions", stats.evictions.longValue());
-    if (maxRamBytes != Long.MAX_VALUE)  {
-      lst.add("cumulative_evictionsRamUsage", stats.evictionsRamUsage.longValue());
-    }
-    
-    return lst;
+    });
+    manager.registerGauge(this, registryName, cacheMap, true, scope, getCategory().toString());
+  }
+
+  // for unit tests only
+  MetricsMap getMetricsMap() {
+    return cacheMap;
+  }
+
+  @Override
+  public MetricRegistry getMetricRegistry() {
+    return registry;
   }
 
   @Override
   public String toString() {
-    return name() + getStatistics().toString();
+    return name() + cacheMap != null ? cacheMap.getValue().toString() : "";
   }
 
   @Override