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 2016/12/27 15:30:12 UTC

[3/4] lucene-solr:branch_6x: Cumulative patch from master, originally developed on branch 'feature/metrics', which brings the following issues: * SOLR-4735: Improve Solr metrics reporting * SOLR-9812: Implement /admin/metrics API * SOLR-9805: Use metrics

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/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
new file mode 100644
index 0000000..9f126ed
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -0,0 +1,669 @@
+/*
+ * 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 java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.SharedMetricRegistries;
+import com.codahale.metrics.Timer;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrResourceLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class maintains a repository of named {@link MetricRegistry} instances, and provides several
+ * helper methods for managing various aspects of metrics reporting:
+ * <ul>
+ *   <li>registry creation, clearing and removal,</li>
+ *   <li>creation of most common metric implementations,</li>
+ *   <li>management of {@link SolrMetricReporter}-s specific to a named registry.</li>
+ * </ul>
+ * {@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.
+ * <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
+ * 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>
+ */
+public class SolrMetricManager {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /** Common prefix for all registry names that Solr uses. */
+  public static final String REGISTRY_NAME_PREFIX = "solr.";
+
+  /** 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();
+
+  /** 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();
+
+  private final ConcurrentMap<String, MetricRegistry> registries = new ConcurrentHashMap<>();
+
+  private final Map<String, Map<String, SolrMetricReporter>> reporters = new HashMap<>();
+
+  private final Lock reportersLock = new ReentrantLock();
+
+  public SolrMetricManager() { }
+
+  /**
+   * An implementation of {@link MetricFilter} that selects metrics
+   * with names that start with a prefix.
+   */
+  public static class PrefixFilter implements MetricFilter {
+    private final String prefix;
+    private final Set<String> matched = new HashSet<>();
+
+    /**
+     * Create a filter that uses the provided prefix.
+     * @param prefix prefix to use, must not be null. If empty then any
+     *               name will match.
+     */
+    public PrefixFilter(String prefix) {
+      Objects.requireNonNull(prefix);
+      this.prefix = prefix;
+    }
+
+    @Override
+    public boolean matches(String name, Metric metric) {
+      if (prefix.isEmpty()) {
+        matched.add(name);
+        return true;
+      }
+      if (name.startsWith(prefix)) {
+        matched.add(name);
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    /**
+     * Return the set of names that matched this filter.
+     * @return matching names
+     */
+    public Set<String> getMatched() {
+      return Collections.unmodifiableSet(matched);
+    }
+
+    /**
+     * Clear the set of names that matched.
+     */
+    public void reset() {
+      matched.clear();
+    }
+  }
+
+  /**
+   * Return a set of existing registry names.
+   */
+  public Set<String> registryNames() {
+    Set<String> set = new HashSet<>();
+    set.addAll(registries.keySet());
+    set.addAll(SharedMetricRegistries.names());
+    return Collections.unmodifiableSet(set);
+  }
+
+  /**
+   * Check for predefined shared registry names. This compares the input name
+   * with normalized and possibly overriden names of predefined shared registries -
+   * {@link #JVM_REGISTRY} and {@link #JETTY_REGISTRY}.
+   * @param registry already normalized and possibly overriden name
+   * @return true if the name matches one of shared registries
+   */
+  private static boolean isSharedRegistry(String registry) {
+    if (overridableRegistryName(JETTY_REGISTRY).equals(registry) ||
+        overridableRegistryName(JVM_REGISTRY).equals(registry)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Get (or create if not present) a named registry
+   * @param registry name of the registry
+   * @return existing or newly created registry
+   */
+  public MetricRegistry registry(String registry) {
+    registry = overridableRegistryName(registry);
+    if (isSharedRegistry(registry)) {
+      return SharedMetricRegistries.getOrCreate(registry);
+    } else {
+      final MetricRegistry existing = registries.get(registry);
+      if (existing == null) {
+        final MetricRegistry created = new MetricRegistry();
+        final MetricRegistry raced = registries.putIfAbsent(registry, created);
+        if (raced == null) {
+          return created;
+        } else {
+          return raced;
+        }
+      } else {
+        return existing;
+      }
+    }
+  }
+
+  /**
+   * Remove a named registry.
+   * @param registry name of the registry to remove
+   */
+  public void removeRegistry(String registry) {
+    // close any reporters for this registry first
+    closeReporters(registry);
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    if (isSharedRegistry(registry)) {
+      SharedMetricRegistries.remove(registry);
+    } else {
+      registries.remove(registry);
+    }
+  }
+
+  /**
+   * Move all matching metrics from one registry to another. This is useful eg. during
+   * {@link org.apache.solr.core.SolrCore} rename or swap operations.
+   * @param fromRegistry source registry
+   * @param toRegistry target registry
+   * @param filter optional {@link MetricFilter} to select what metrics to move. If null
+   *               then all metrics will be moved.
+   */
+  public void moveMetrics(String fromRegistry, String toRegistry, MetricFilter filter) {
+    MetricRegistry from = registry(fromRegistry);
+    MetricRegistry to = registry(toRegistry);
+    if (from == to) {
+      return;
+    }
+    if (filter == null) {
+      to.registerAll(from);
+      from.removeMatching(MetricFilter.ALL);
+    } else {
+      for (Map.Entry<String, Metric> entry : from.getMetrics().entrySet()) {
+        if (filter.matches(entry.getKey(), entry.getValue())) {
+          to.register(entry.getKey(), entry.getValue());
+        }
+      }
+      from.removeMatching(filter);
+    }
+  }
+
+  /**
+   * Register all metrics in the provided {@link MetricSet}, optionally skipping those that
+   * already exist.
+   * @param registry registry name
+   * @param metrics metric set to register
+   * @param force if true then already existing metrics with the same name will be replaced.
+   *                     When false and a metric with the same name already exists an exception
+   *                     will be thrown.
+   * @param metricPath (optional) additional top-most metric name path elements
+   * @throws Exception if a metric with this name already exists.
+   */
+  public void registerAll(String registry, MetricSet metrics, boolean force, String... metricPath) throws Exception {
+    MetricRegistry metricRegistry = registry(registry);
+    synchronized (metricRegistry) {
+      Map<String, Metric> existingMetrics = metricRegistry.getMetrics();
+      for (Map.Entry<String, Metric> entry : metrics.getMetrics().entrySet()) {
+        String fullName = mkName(entry.getKey(), metricPath);
+        if (force && existingMetrics.containsKey(fullName)) {
+          metricRegistry.remove(fullName);
+        }
+        metricRegistry.register(fullName, entry.getValue());
+      }
+    }
+  }
+
+  /**
+   * Remove all metrics from a specified registry.
+   * @param registry registry name
+   */
+  public void clearRegistry(String registry) {
+    registry(registry).removeMatching(MetricFilter.ALL);
+  }
+
+  /**
+   * Remove some metrics from a named registry
+   * @param registry registry name
+   * @param metricPath (optional) top-most metric name path elements. If empty then
+   *        this is equivalent to calling {@link #clearRegistry(String)},
+   *        otherwise non-empty elements will be joined using dotted notation
+   *        to form a fully-qualified prefix. Metrics with names that start
+   *        with the prefix will be removed.
+   * @return set of metrics names that have been removed.
+   */
+  public Set<String> clearMetrics(String registry, String... metricPath) {
+    PrefixFilter filter;
+    if (metricPath == null || metricPath.length == 0) {
+      filter = new PrefixFilter("");
+    } else {
+      String prefix = MetricRegistry.name("", metricPath);
+      filter = new PrefixFilter(prefix);
+    }
+    registry(registry).removeMatching(filter);
+    return filter.getMatched();
+  }
+
+  /**
+   * Create or get an existing named {@link Meter}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @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));
+  }
+
+  /**
+   * Create or get an existing named {@link Timer}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @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));
+  }
+
+  /**
+   * Create or get an existing named {@link Counter}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @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));
+  }
+
+  /**
+   * Create or get an existing named {@link Histogram}
+   * @param registry registry name
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   using dotted notation
+   * @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));
+  }
+
+  /**
+   * Register an instance of {@link Metric}.
+   * @param registry registry name
+   * @param metric metric instance
+   * @param force if true then an already existing metric with the same name will be replaced.
+   *                     When false and a metric with the same name already exists an exception
+   *                     will be thrown.
+   * @param metricName metric name, either final name or a fully-qualified name
+   *                   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) {
+    MetricRegistry metricRegistry = registry(registry);
+    String fullName = mkName(metricName, metricPath);
+    synchronized (metricRegistry) {
+      if (force && metricRegistry.getMetrics().containsKey(fullName)) {
+        metricRegistry.remove(fullName);
+      }
+      metricRegistry.register(fullName, metric);
+    }
+  }
+
+
+
+  /**
+   * This method creates a hierarchical name with arbitrary levels of hierarchy
+   * @param name the final segment of the name, must not be null or empty.
+   * @param path optional path segments, starting from the top level. Empty or null
+   *             segments will be skipped.
+   * @return fully-qualified name using dotted notation, with all valid hierarchy
+   * segments prepended to the name.
+   */
+  public static String mkName(String name, String... path) {
+    if (name == null || name.isEmpty()) {
+      throw new IllegalArgumentException("name must not be empty");
+    }
+    if (path == null || path.length == 0) {
+      return name;
+    } else {
+      StringBuilder sb = new StringBuilder();
+      for (String s : path) {
+        if (s == null || s.isEmpty()) {
+          continue;
+        }
+        if (sb.length() > 0) {
+          sb.append('.');
+        }
+        sb.append(s);
+      }
+      if (sb.length() > 0) {
+        sb.append('.');
+      }
+      sb.append(name);
+      return sb.toString();
+    }
+  }
+
+  /**
+   * Allows named registries to be renamed using System properties.
+   * This would be mostly be useful if you want to combine the metrics from a few registries for a single
+   * reporter.
+   * <p>For example, in order to collect metrics from related cores in a single registry you could specify
+   * the following system properties:</p>
+   * <pre>
+   *   ... -Dsolr.core.collection1=solr.core.allCollections -Dsolr.core.collection2=solr.core.allCollections
+   * </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.
+   * @param registry The name of the registry
+   * @return A potentially overridden (via System properties) registry name
+   */
+  public static String overridableRegistryName(String registry) {
+    String fqRegistry = enforcePrefix(registry);
+    return enforcePrefix(System.getProperty(fqRegistry,fqRegistry));
+  }
+
+  /**
+   * Enforces the leading {@link #REGISTRY_NAME_PREFIX} in a name.
+   * @param name input name, possibly without the prefix
+   * @return original name if it contained the prefix, or the
+   * input name with the prefix prepended.
+   */
+  public static String enforcePrefix(String name) {
+    if (name.startsWith(REGISTRY_NAME_PREFIX)) {
+      return name;
+    } else {
+      return new StringBuilder(REGISTRY_NAME_PREFIX).append(name).toString();
+    }
+  }
+
+  /**
+   * Helper method to construct a properly prefixed registry name based on the group.
+   * @param group reporting group
+   * @param names optional child elements of the registry name. If exactly one element is provided
+   *              and it already contains the required prefix and group name then this value will be used,
+   *              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) {
+    String fullName;
+    String prefix = REGISTRY_NAME_PREFIX + group.toString() + ".";
+    // check for existing prefix and group
+    if (names != null && names.length > 0 && names[0] != null && names[0].startsWith(prefix)) {
+      // assume the first segment already was expanded
+      if (names.length > 1) {
+        String[] newNames = new String[names.length - 1];
+        System.arraycopy(names, 1, newNames, 0, newNames.length);
+        fullName = MetricRegistry.name(names[0], newNames);
+      } else {
+        fullName = MetricRegistry.name(names[0]);
+      }
+    } else {
+      fullName = MetricRegistry.name(group.toString(), names);
+    }
+    return overridableRegistryName(fullName);
+  }
+
+  // reporter management
+
+  /**
+   * Create and register {@link SolrMetricReporter}-s specific to a {@link org.apache.solr.core.SolrInfoMBean.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
+   * the list. If both attributes are present then only "group" attribute will be processed.
+   * @param pluginInfos plugin configurations
+   * @param loader resource loader
+   * @param group selected group, not null
+   * @param registryNames optional child registry name elements
+   */
+  public void loadReporters(PluginInfo[] pluginInfos, SolrResourceLoader loader, SolrInfoMBean.Group group, String... registryNames) {
+    if (pluginInfos == null || pluginInfos.length == 0) {
+      return;
+    }
+    String registryName = getRegistryName(group, registryNames);
+    for (PluginInfo info : pluginInfos) {
+      String target = info.attributes.get("group");
+      if (target == null) { // no "group"
+        target = info.attributes.get("registry");
+        if (target != null) {
+          String[] targets = target.split("[\\s,]+");
+          boolean found = false;
+          for (String t : targets) {
+            t = overridableRegistryName(t);
+            if (registryName.equals(t)) {
+              found = true;
+              break;
+            }
+          }
+          if (!found) {
+            continue;
+          }
+        } else {
+          // neither group nor registry specified.
+          // always register this plugin for all groups and registries
+        }
+      } else { // check groups
+        String[] targets = target.split("[\\s,]+");
+        boolean found = false;
+        for (String t : targets) {
+          if (group.toString().equals(t)) {
+            found = true;
+            break;
+          }
+        }
+        if (!found) {
+          continue;
+        }
+      }
+      try {
+        loadReporter(registryName, loader, info);
+      } catch (Exception e) {
+        log.warn("Error loading metrics reporter, plugin info: " + info, e);
+      }
+    }
+  }
+
+  /**
+   * Create and register an instance of {@link SolrMetricReporter}.
+   * @param registry reporter is associated with this registry
+   * @param loader loader to use when creating an instance of the reporter
+   * @param pluginInfo plugin configuration. Plugin "name" and "class" attributes are required.
+   * @throws Exception if any argument is missing or invalid
+   */
+  public void loadReporter(String registry, SolrResourceLoader loader, PluginInfo pluginInfo) throws Exception {
+    if (registry == null || pluginInfo == null || pluginInfo.name == null || pluginInfo.className == null) {
+      throw new IllegalArgumentException("loadReporter called with missing arguments: " +
+          "registry=" + registry + ", loader=" + loader + ", pluginInfo=" + pluginInfo);
+    }
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    SolrMetricReporter reporter = loader.newInstance(
+        pluginInfo.className,
+        SolrMetricReporter.class,
+        new String[0],
+        new Class[] { SolrMetricManager.class, String.class },
+        new Object[] { this, registry }
+    );
+    try {
+      reporter.init(pluginInfo);
+    } catch (IllegalStateException e) {
+      throw new IllegalArgumentException("reporter init failed: " + pluginInfo, e);
+    }
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        throw new Exception("Could not obtain lock to modify reporters registry: " + registry);
+      }
+    } catch (InterruptedException e) {
+      throw new Exception("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+    }
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
+      if (perRegistry == null) {
+        perRegistry = new HashMap<>();
+        reporters.put(registry, perRegistry);
+      }
+      SolrMetricReporter oldReporter = perRegistry.get(pluginInfo.name);
+      if (oldReporter != null) { // close it
+        log.info("Replacing existing reporter '" + pluginInfo.name + "' in registry '" + registry + "': " + oldReporter.toString());
+        oldReporter.close();
+      }
+      perRegistry.put(pluginInfo.name, reporter);
+
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+
+  /**
+   * Close and unregister a named {@link SolrMetricReporter} for a registry.
+   * @param registry registry name
+   * @param name reporter name
+   * @return true if a named reporter existed and was closed.
+   */
+  public boolean closeReporter(String registry, String name) {
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        log.warn("Could not obtain lock to modify reporters registry: " + registry);
+        return false;
+      }
+    } catch (InterruptedException e) {
+      log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+      return false;
+    }
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
+      if (perRegistry == null) {
+        return false;
+      }
+      SolrMetricReporter reporter = perRegistry.remove(name);
+      if (reporter == null) {
+        return false;
+      }
+      try {
+        reporter.close();
+      } catch (Exception e) {
+        log.warn("Error closing metric reporter, registry=" + registry + ", name=" + name, e);
+      }
+      return true;
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+
+  /**
+   * Close and unregister all {@link SolrMetricReporter}-s for a registry.
+   * @param registry registry name
+   * @return names of closed reporters
+   */
+  public Set<String> closeReporters(String registry) {
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        log.warn("Could not obtain lock to modify reporters registry: " + registry);
+        return Collections.emptySet();
+      }
+    } catch (InterruptedException e) {
+      log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+      return Collections.emptySet();
+    }
+    log.info("Closing metric reporters for: " + registry);
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.remove(registry);
+      if (perRegistry != null) {
+        for (SolrMetricReporter reporter : perRegistry.values()) {
+          try {
+            reporter.close();
+          } catch (IOException ioe) {
+            log.warn("Exception closing reporter " + reporter, ioe);
+          }
+        }
+        return perRegistry.keySet();
+      } else {
+        return Collections.emptySet();
+      }
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+
+  /**
+   * Get a map of reporters for a registry. Keys are reporter names, values are reporter instances.
+   * @param registry registry name
+   * @return map of reporters and their names, may be empty but never null
+   */
+  public Map<String, SolrMetricReporter> getReporters(String registry) {
+    // make sure we use a name with prefix, with overrides
+    registry = overridableRegistryName(registry);
+    try {
+      if (!reportersLock.tryLock(10, TimeUnit.SECONDS)) {
+        log.warn("Could not obtain lock to modify reporters registry: " + registry);
+        return Collections.emptyMap();
+      }
+    } catch (InterruptedException e) {
+      log.warn("Interrupted while trying to obtain lock to modify reporters registry: " + registry);
+      return Collections.emptyMap();
+    }
+    try {
+      Map<String, SolrMetricReporter> perRegistry = reporters.get(registry);
+      if (perRegistry == null) {
+        return Collections.emptyMap();
+      } else {
+        // defensive copy - the original map may change after we release the lock
+        return Collections.unmodifiableMap(new HashMap<>(perRegistry));
+      }
+    } finally {
+      reportersLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
new file mode 100644
index 0000000..472931f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricProducer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.metrics;
+
+import java.util.Collection;
+
+import org.apache.solr.core.SolrInfoMBean;
+
+/**
+ * Extension of {@link SolrInfoMBean} for use by objects that
+ * expose metrics through {@link SolrCoreMetricManager}.
+ */
+public interface SolrMetricProducer extends SolrInfoMBean {
+
+  /**
+   * Initializes metrics specific to this producer
+   * @param manager an instance of {@link SolrMetricManager}
+   * @param registry registry name where metrics are registered
+   * @param scope scope of the metrics (eg. handler name) to separate metrics of
+   *              instances of the same component executing in different contexts
+   * @return registered (or existing) unqualified names of metrics specific to this producer.
+   */
+  Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/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
new file mode 100644
index 0000000..ff2d3fc
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricReporter.java
@@ -0,0 +1,83 @@
+/*
+ * 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 java.io.Closeable;
+
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.util.SolrPluginUtils;
+import org.apache.solr.util.plugin.PluginInfoInitialized;
+
+/**
+ * Interface for 'pluggable' metric reporters.
+ */
+public abstract class SolrMetricReporter implements Closeable, PluginInfoInitialized {
+
+  protected final String registryName;
+  protected final SolrMetricManager metricManager;
+  protected PluginInfo pluginInfo;
+
+  /**
+   * Create a reporter for metrics managed in a named registry.
+   * @param registryName registry to use, one of registries managed by
+   *                     {@link SolrMetricManager}
+   */
+  protected SolrMetricReporter(SolrMetricManager metricManager, String registryName) {
+    this.registryName = registryName;
+    this.metricManager = metricManager;
+  }
+
+  /**
+   * Initializes a {@link SolrMetricReporter} with the plugin's configuration.
+   *
+   * @param pluginInfo the plugin's configuration
+   */
+  @SuppressWarnings("unchecked")
+  public void init(PluginInfo pluginInfo) {
+    if (pluginInfo != null) {
+      this.pluginInfo = pluginInfo.copy();
+      if (this.pluginInfo.initArgs != null) {
+        SolrPluginUtils.invokeSetters(this, this.pluginInfo.initArgs);
+      }
+    }
+    validate();
+  }
+
+  /**
+   * Get the effective {@link PluginInfo} instance that was used for
+   * initialization of this plugin.
+   * @return plugin info, or null if not yet initialized.
+   */
+  public PluginInfo getPluginInfo() {
+    return pluginInfo;
+  }
+
+  /**
+   * Validates that the reporter has been correctly configured.
+   *
+   * @throws IllegalStateException if the reporter is not properly configured
+   */
+  protected abstract void validate() throws IllegalStateException;
+
+  @Override
+  public String toString() {
+    return getClass().getName() + "{" +
+        "registryName='" + registryName + '\'' +
+        ", pluginInfo=" + pluginInfo +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/metrics/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/package-info.java b/solr/core/src/java/org/apache/solr/metrics/package-info.java
new file mode 100644
index 0000000..a007b8b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * The {@link org.apache.solr.metrics.SolrCoreMetricManager} is responsible for
+ * collecting metrics from {@link org.apache.solr.metrics.SolrMetricProducer}'s
+ * and exposing metrics to {@link org.apache.solr.metrics.SolrMetricReporter}'s.
+ */
+package org.apache.solr.metrics;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/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
new file mode 100644
index 0000000..47fbf11
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/SolrJmxReporter.java
@@ -0,0 +1,284 @@
+/*
+ * 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 javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.util.Locale;
+
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.ObjectNameFactory;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.SolrMetricInfo;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.apache.solr.util.JmxUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link SolrMetricReporter} that finds (or creates) a MBeanServer from
+ * the given configuration and registers metrics to it with JMX.
+ */
+public class SolrJmxReporter extends SolrMetricReporter {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private String domain;
+  private String agentId;
+  private String serviceUrl;
+
+  private JmxReporter reporter;
+  private MBeanServer mBeanServer;
+
+  /**
+   * Creates a new instance of {@link SolrJmxReporter}.
+   *
+   * @param registryName name of the registry to report
+   */
+  public SolrJmxReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+    setDomain(registryName);
+  }
+
+  /**
+   * Initializes the reporter by finding (or creating) a MBeanServer
+   * and registering the metricManager's metric registry.
+   *
+   * @param pluginInfo the configuration for the reporter
+   */
+  @Override
+  public synchronized void init(PluginInfo pluginInfo) {
+    super.init(pluginInfo);
+
+    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.",
+          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) {
+      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);
+    }
+
+    if (mBeanServer == null) {
+      log.warn("No JMX server found. Not exposing Solr metrics.");
+      return;
+    }
+
+    JmxObjectNameFactory jmxObjectNameFactory = new JmxObjectNameFactory(pluginInfo.name, domain);
+
+    reporter = JmxReporter.forRegistry(metricManager.registry(registryName))
+                          .registerWith(mBeanServer)
+                          .inDomain(domain)
+                          .createsObjectNamesWith(jmxObjectNameFactory)
+                          .build();
+    reporter.start();
+
+    log.info("JMX monitoring enabled at server: " + mBeanServer);
+  }
+
+  /**
+   * Stops the reporter from publishing metrics.
+   */
+  @Override
+  public synchronized void close() {
+    if (reporter != null) {
+      reporter.close();
+      reporter = null;
+    }
+  }
+
+  /**
+   * Validates that the reporter has been correctly configured.
+   * Note that all configurable arguments are currently optional.
+   *
+   * @throws IllegalStateException if the reporter is not properly configured
+   */
+  @Override
+  protected void validate() throws IllegalStateException {
+    // Nothing to validate
+  }
+
+  /**
+   * Sets the domain with which MBeans are published. If none is set,
+   * the domain defaults to the name of the core.
+   *
+   * @param domain the domain
+   */
+  public void setDomain(String domain) {
+    if (domain != null) {
+      this.domain = domain;
+    } else {
+      this.domain = registryName;
+    }
+  }
+
+  /**
+   * Sets the service url for a JMX server.
+   * Note that this configuration is optional.
+   *
+   * @param serviceUrl the service url
+   */
+  public void setServiceUrl(String serviceUrl) {
+    this.serviceUrl = serviceUrl;
+  }
+
+  /**
+   * Sets the agent id for a JMX server.
+   * Note that this configuration is optional.
+   *
+   * @param agentId the agent id
+   */
+  public void setAgentId(String agentId) {
+    this.agentId = agentId;
+  }
+
+  /**
+   * Retrieves the reporter's MBeanServer.
+   *
+   * @return the reporter's MBeanServer
+   */
+  public MBeanServer getMBeanServer() {
+    return mBeanServer;
+  }
+
+  @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);
+  }
+
+  /**
+   * Factory to create MBean names for a given metric.
+   */
+  private static class JmxObjectNameFactory implements ObjectNameFactory {
+
+    private final String domain;
+    private final String[] subdomains;
+    private final String reporterName;
+
+    JmxObjectNameFactory(String reporterName, String domain) {
+      this.reporterName = reporterName;
+      this.domain = domain;
+      this.subdomains = domain.split("\\.");
+    }
+
+    /**
+     * Create a hierarchical name of a metric.
+     *
+     * @param type    metric class, eg. "counters"
+     * @param currentDomain  JMX domain
+     * @param name    metric name
+     */
+    @Override
+    public ObjectName createName(String type, String currentDomain, String name) {
+      SolrMetricInfo metricInfo = SolrMetricInfo.of(name);
+
+      // 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
+      // that seems to have a mind of its own...
+      StringBuilder sb = new StringBuilder();
+      if (domain.equals(currentDomain)) {
+        if (subdomains != null && subdomains.length > 1) {
+          // use only first segment as domain
+          sb.append(subdomains[0]);
+          sb.append(':');
+          // use remaining segments as properties
+          for (int i = 1; i < subdomains.length; i++) {
+            if (i > 1) {
+              sb.append(',');
+            }
+            sb.append("dom");
+            sb.append(String.valueOf(i));
+            sb.append('=');
+            sb.append(subdomains[i]);
+          }
+          sb.append(','); // separate from other properties
+        } else {
+          sb.append(currentDomain);
+          sb.append(':');
+        }
+      } else {
+        sb.append(currentDomain);
+        sb.append(':');
+      }
+      sb.append("reporter=");
+      sb.append(reporterName);
+      sb.append(',');
+      if (metricInfo != null) {
+        sb.append("category=");
+        sb.append(metricInfo.category.toString());
+        sb.append(",scope=");
+        sb.append(metricInfo.scope);
+        // we could also split by type, but don't call it 'type' :)
+        // sb.append(",class=");
+        //sb.append(type);
+        sb.append(",name=");
+        sb.append(metricInfo.name);
+      } else {
+        // make dotted names into hierarchies
+        String[] path = name.split("\\.");
+        for (int i = 0; i < path.length - 1; i++) {
+          if (i > 0) {
+            sb.append(',');
+          }
+          sb.append("name"); sb.append(String.valueOf(i));
+          sb.append('=');
+          sb.append(path[i]);
+        }
+        if (path.length > 1) {
+          sb.append(',');
+        }
+        // split by type
+        // sb.append("class=");
+        // sb.append(type);
+        sb.append("name=");
+        sb.append(path[path.length - 1]);
+      }
+
+      ObjectName objectName;
+
+      try {
+        objectName = new ObjectName(sb.toString());
+      } catch (MalformedObjectNameException e) {
+        throw new RuntimeException(sb.toString(), e);
+      }
+
+      return objectName;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java b/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java
new file mode 100644
index 0000000..5ad7a80
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package houses 'pluggable' metric reporters that
+ * inherit from the {@link org.apache.solr.metrics.SolrMetricReporter} class.
+ */
+package org.apache.solr.metrics.reporters;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
index d0e5c9a..425be38 100644
--- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
+++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java
@@ -46,6 +46,7 @@ public interface PermissionNameProvider {
     SCHEMA_EDIT_PERM("schema-edit", "*"),
     SECURITY_EDIT_PERM("security-edit", null),
     SECURITY_READ_PERM("security-read", null),
+    METRICS_READ_PERM("metrics-read", null),
     ALL("all", unmodifiableSet(new HashSet<>(asList("*", null))))
     ;
     final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index e8c4657..a411bb3 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.servlet;
 
+import javax.management.MBeanServer;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
@@ -33,6 +34,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.time.Instant;
@@ -45,6 +47,11 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.codahale.metrics.jvm.BufferPoolMetricSet;
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
 import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
@@ -58,8 +65,11 @@ import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.OperatingSystemMetricSet;
+import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
@@ -157,6 +167,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       this.cores = createCoreContainer(solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome),
                                        extraProperties);
       this.httpClient = cores.getUpdateShardHandler().getHttpClient();
+      setupJvmMetrics();
       log.debug("user.dir=" + System.getProperty("user.dir"));
     }
     catch( Throwable t ) {
@@ -171,6 +182,22 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     log.trace("SolrDispatchFilter.init() done");
   }
 
+  private void setupJvmMetrics()  {
+    MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+    SolrMetricManager metricManager = cores.getMetricManager();
+    try {
+      String registry = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm);
+      metricManager.registerAll(registry, new BufferPoolMetricSet(platformMBeanServer), true, "buffers");
+      metricManager.registerAll(registry, new ClassLoadingGaugeSet(), true, "classes");
+      metricManager.registerAll(registry, new OperatingSystemMetricSet(platformMBeanServer), true, "os");
+      metricManager.registerAll(registry, new GarbageCollectorMetricSet(), true, "gc");
+      metricManager.registerAll(registry, new MemoryUsageGaugeSet(), true, "memory");
+      metricManager.registerAll(registry, new ThreadStatesGaugeSet(), true, "threads"); // todo should we use CachedThreadStatesGaugeSet instead?
+    } catch (Exception e) {
+      log.warn("Error registering JVM metrics", e);
+    }
+  }
+
   private void logWelcomeBanner() {
     log.info(" ___      _       Welcome to Apache Solr\u2122 version {}", solrVersion());
     log.info("/ __| ___| |_ _   Starting in {} mode on port {}", isCloudMode() ? "cloud" : "standalone", getSolrPort());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/util/JmxUtil.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/JmxUtil.java b/solr/core/src/java/org/apache/solr/util/JmxUtil.java
new file mode 100644
index 0000000..02a070d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/JmxUtil.java
@@ -0,0 +1,78 @@
+/*
+ * 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.util;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.remote.JMXConnectorServer;
+import javax.management.remote.JMXConnectorServerFactory;
+import javax.management.remote.JMXServiceURL;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Utility methods to find a MBeanServer.
+ *
+ * This was factored out from {@link org.apache.solr.core.JmxMonitoredMap}
+ * and can eventually replace the logic used there.
+ */
+public final class JmxUtil {
+
+  /**
+   * Retrieve the first MBeanServer found.
+   *
+   * @return the first MBeanServer found
+   */
+  public static MBeanServer findFirstMBeanServer() {
+    return findMBeanServerForAgentId(null);
+  }
+
+  /**
+   * Find a MBeanServer given a service url.
+   *
+   * @param serviceUrl the service url
+   * @return a MBeanServer
+   */
+  public static MBeanServer findMBeanServerForServiceUrl(String serviceUrl) throws IOException {
+    if (serviceUrl == null) {
+      return null;
+    }
+
+    MBeanServer server = MBeanServerFactory.newMBeanServer();
+    JMXConnectorServer connector = JMXConnectorServerFactory
+        .newJMXConnectorServer(new JMXServiceURL(serviceUrl), null, server);
+    connector.start();
+
+    return server;
+  }
+
+  /**
+   * Find a MBeanServer given an agent id.
+   *
+   * @param agentId the agent id
+   * @return a MBeanServer
+   */
+  public static MBeanServer findMBeanServerForAgentId(String agentId) {
+    List<MBeanServer> servers = MBeanServerFactory.findMBeanServer(agentId);
+    if (servers == null || servers.isEmpty()) {
+      return null;
+    }
+
+    return servers.get(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
new file mode 100644
index 0000000..62f5776
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/stats/MetricUtils.java
@@ -0,0 +1,144 @@
+/*
+ * 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.util.stats;
+
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Metrics specific utility functions.
+ */
+public class MetricUtils {
+
+  /**
+   * Adds metrics from a Timer to a NamedList, using well-known names.
+   * @param lst The NamedList to add the metrics data to
+   * @param timer The Timer to extract the metrics from
+   */
+  public static void addMetrics(NamedList<Object> lst, Timer timer) {
+    Snapshot snapshot = timer.getSnapshot();
+    lst.add("avgRequestsPerSecond", timer.getMeanRate());
+    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
+    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
+    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+  }
+
+  /**
+   * Converts a double representing nanoseconds to a double representing milliseconds.
+   *
+   * @param ns the amount of time in nanoseconds
+   * @return the amount of time in milliseconds
+   */
+  static double nsToMs(double ns) {
+    return ns / TimeUnit.MILLISECONDS.toNanos(1);
+  }
+
+  /**
+   * Returns a NamedList respresentation of the given metric registry. Only those metrics
+   * are converted to NamedList which match at least one of the given MetricFilter instances.
+   *
+   * @param registry      the {@link MetricRegistry} to be converted to NamedList
+   * @param metricFilters a list of {@link MetricFilter} instances
+   * @return a {@link NamedList}
+   */
+  public static NamedList toNamedList(MetricRegistry registry, List<MetricFilter> metricFilters) {
+    NamedList response = new NamedList();
+    Map<String, Metric> metrics = registry.getMetrics();
+    SortedSet<String> names = registry.getNames();
+    names.stream().filter(s -> metricFilters.stream().anyMatch(metricFilter -> metricFilter.matches(s, metrics.get(s)))).forEach(n -> {
+      Metric metric = metrics.get(n);
+      if (metric instanceof Counter) {
+        Counter counter = (Counter) metric;
+        response.add(n, counterToNamedList(counter));
+      } else if (metric instanceof Gauge) {
+        Gauge gauge = (Gauge) metric;
+        response.add(n, gaugeToNamedList(gauge));
+      } else if (metric instanceof Meter) {
+        Meter meter = (Meter) metric;
+        response.add(n, meterToNamedList(meter));
+      } else if (metric instanceof Timer) {
+        Timer timer = (Timer) metric;
+        response.add(n, timerToNamedList(timer));
+      } else if (metric instanceof Histogram) {
+        Histogram histogram = (Histogram) metric;
+        response.add(n, histogramToNamedList(histogram));
+      }
+    });
+    return response;
+  }
+
+  static NamedList histogramToNamedList(Histogram histogram) {
+    NamedList response = new NamedList();
+    Snapshot snapshot = histogram.getSnapshot();
+    response.add("requests", histogram.getCount());
+    response.add("minTime", nsToMs(snapshot.getMin()));
+    response.add("maxTime", nsToMs(snapshot.getMax()));
+    response.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
+    response.add("medianRequestTime", nsToMs(snapshot.getMedian()));
+    response.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
+    response.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
+    response.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
+    response.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
+    return response;
+  }
+
+  static NamedList timerToNamedList(Timer timer) {
+    NamedList response = new NamedList();
+    addMetrics(response, timer);
+    return response;
+  }
+
+  static NamedList meterToNamedList(Meter meter) {
+    NamedList response = new NamedList();
+    response.add("requests", meter.getCount());
+    response.add("avgRequestsPerSecond", meter.getMeanRate());
+    response.add("1minRateRequestsPerSecond", meter.getOneMinuteRate());
+    response.add("5minRateRequestsPerSecond", meter.getFiveMinuteRate());
+    response.add("15minRateRequestsPerSecond", meter.getFifteenMinuteRate());
+    return response;
+  }
+
+  static NamedList gaugeToNamedList(Gauge gauge) {
+    NamedList response = new NamedList();
+    response.add("value", gauge.getValue());
+    return response;
+  }
+
+  static NamedList counterToNamedList(Counter counter) {
+    NamedList response = new NamedList();
+    response.add("requests", counter.getCount());
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java b/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
deleted file mode 100644
index 243c1ee..0000000
--- a/solr/core/src/java/org/apache/solr/util/stats/TimerUtils.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.util.stats;
-
-import java.util.concurrent.TimeUnit;
-
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
-import org.apache.solr.common.util.NamedList;
-
-/**
- * Solr specific {@link Timer} utility functions.
- */
-public class TimerUtils {
-
-  /**
-   * Adds metrics from a Timer to a NamedList, using well-known names.
-   * @param lst The NamedList to add the metrics data to
-   * @param timer The Timer to extract the metrics from
-   */
-  public static void addMetrics(NamedList<Object> lst, Timer timer) {
-    Snapshot snapshot = timer.getSnapshot();
-    lst.add("avgRequestsPerSecond", timer.getMeanRate());
-    lst.add("5minRateRequestsPerSecond", timer.getFiveMinuteRate());
-    lst.add("15minRateRequestsPerSecond", timer.getFifteenMinuteRate());
-    lst.add("avgTimePerRequest", nsToMs(snapshot.getMean()));
-    lst.add("medianRequestTime", nsToMs(snapshot.getMedian()));
-    lst.add("75thPcRequestTime", nsToMs(snapshot.get75thPercentile()));
-    lst.add("95thPcRequestTime", nsToMs(snapshot.get95thPercentile()));
-    lst.add("99thPcRequestTime", nsToMs(snapshot.get99thPercentile()));
-    lst.add("999thPcRequestTime", nsToMs(snapshot.get999thPercentile()));
-  }
-
-  /**
-   * Converts a double representing nanoseconds to a double representing milliseconds.
-   *
-   * @param ns the amount of time in nanoseconds
-   * @return the amount of time in milliseconds
-   */
-  static double nsToMs(double ns) {
-    return ns / TimeUnit.MILLISECONDS.toNanos(1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/test-files/solr/solr-metricreporter.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-metricreporter.xml b/solr/core/src/test-files/solr/solr-metricreporter.xml
new file mode 100644
index 0000000..cd8d737
--- /dev/null
+++ b/solr/core/src/test-files/solr/solr-metricreporter.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<solr>
+ <metrics>
+  <!-- this reporter doesn't specify 'group' or 'registry', it will be instantiated for any group. -->
+  <reporter name="universal" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- this reporter will be created for both "node" and "core" groups -->
+  <reporter name="multigroup" group="node, core" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- this reporter will be created for both "node" and "core.collection1" registries -->
+  <reporter name="multiregistry" registry="node, core.collection1" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="jvm" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="jetty" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="http" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter1" group="node" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- core reporters. -->
+  <reporter name="reporter1" group="core" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <reporter name="reporter2" group="core" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+  <!-- reporter for a specific registry -->
+  <reporter name="specific" registry="core.collection1" class="org.apache.solr.metrics.reporters.MockMetricReporter">
+    <str name="configurable">configured</str>
+  </reporter>
+ </metrics>
+</solr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
new file mode 100644
index 0000000..3667285
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/MetricsHandlerTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.admin;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.response.SolrQueryResponse;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test for {@link MetricsHandler}
+ */
+public class MetricsHandlerTest extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void test() throws Exception {
+    MetricsHandler handler = new MetricsHandler(h.getCoreContainer());
+
+    SolrQueryResponse resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json"), resp);
+    NamedList values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    System.out.println(values);
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+    assertNotNull(values.get("solr.http"));
+    assertNotNull(values.get("solr.node"));
+    NamedList nl = (NamedList) values.get("solr.core.collection1");
+    assertNotNull(nl);
+    assertNotNull(nl.get("newSearcherErrors")); // counter type
+    assertNotNull(((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    assertEquals(0L, ((NamedList) nl.get("newSearcherErrors")).get("requests"));
+    nl = (NamedList) values.get("solr.node");
+    assertNotNull(nl.get("cores.loaded")); // int gauge
+    assertEquals(1, ((NamedList) nl.get("cores.loaded")).get("value"));
+    assertNotNull(nl.get("QUERYHANDLER./admin/authorization.clientErrors")); // timer type
+    assertEquals(5, ((NamedList) nl.get("QUERYHANDLER./admin/authorization.clientErrors")).size());
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm,jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "jvm", "group", "jetty"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(2, values.size());
+    assertNotNull(values.get("solr.jetty"));
+    assertNotNull(values.get("solr.jvm"));
+
+    resp = new SolrQueryResponse();
+    handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "counter"), resp);
+    values = resp.getValues();
+    assertNotNull(values.get("metrics"));
+    values = (NamedList) values.get("metrics");
+    assertEquals(1, values.size());
+    assertNotNull(values.get("solr.node"));
+    assertNull(values.get("QUERYHANDLER./admin/authorization.errors")); // this is a timer node
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java b/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java
new file mode 100644
index 0000000..72adc68
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/JvmMetricsTest.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.metrics;
+
+import javax.management.MBeanServer;
+import java.lang.management.ManagementFactory;
+import java.util.Map;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import org.apache.solr.SolrJettyTestBase;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test {@link OperatingSystemMetricSet} and proper JVM metrics registration.
+ */
+public class JvmMetricsTest extends SolrJettyTestBase {
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    createJetty(legacyExampleCollection1SolrHome());
+  }
+
+  @Test
+  public void testOperatingSystemMetricsSet() throws Exception {
+    MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
+    OperatingSystemMetricSet set = new OperatingSystemMetricSet(mBeanServer);
+    Map<String, Metric> metrics = set.getMetrics();
+    assertTrue(metrics.size() > 0);
+    for (String metric : OperatingSystemMetricSet.METRICS) {
+      Gauge<?> gauge = (Gauge<?>)metrics.get(metric);
+      if (gauge == null || gauge.getValue() == null) { // some are optional depending on OS
+        continue;
+      }
+      double value = ((Number)gauge.getValue()).doubleValue();
+      // SystemLoadAverage on Windows may be -1.0
+      assertTrue("unexpected value of " + metric + ": " + value, value >= 0 || value == -1.0);
+    }
+  }
+
+  @Test
+  public void testSetupJvmMetrics() throws Exception {
+    SolrMetricManager metricManager = jetty.getCoreContainer().getMetricManager();
+    Map<String,Metric> metrics = metricManager.registry("solr.jvm").getMetrics();
+    assertTrue(metrics.size() > 0);
+    assertTrue(metrics.toString(), metrics.entrySet().stream().filter(e -> e.getKey().startsWith("buffers.")).count() > 0);
+    assertTrue(metrics.toString(), metrics.entrySet().stream().filter(e -> e.getKey().startsWith("classes.")).count() > 0);
+    assertTrue(metrics.toString(), metrics.entrySet().stream().filter(e -> e.getKey().startsWith("os.")).count() > 0);
+    assertTrue(metrics.toString(), metrics.entrySet().stream().filter(e -> e.getKey().startsWith("gc.")).count() > 0);
+    assertTrue(metrics.toString(), metrics.entrySet().stream().filter(e -> e.getKey().startsWith("memory.")).count() > 0);
+    assertTrue(metrics.toString(), metrics.entrySet().stream().filter(e -> e.getKey().startsWith("threads.")).count() > 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9dde8a30/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
new file mode 100644
index 0000000..65ffb93
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrCoreMetricManagerTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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 java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.apache.solr.schema.FieldType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SolrCoreMetricManagerTest extends SolrTestCaseJ4 {
+  private static final int MAX_ITERATIONS = 100;
+
+  private SolrCoreMetricManager coreMetricManager;
+  private SolrMetricManager metricManager;
+
+  @Before
+  public void beforeTest() throws Exception {
+    initCore("solrconfig-basic.xml", "schema.xml");
+    coreMetricManager = h.getCore().getCoreMetricManager();
+    metricManager = h.getCore().getCoreDescriptor().getCoreContainer().getMetricManager();
+  }
+
+  @After
+  public void afterTest() throws IOException {
+    coreMetricManager.close();
+    assertTrue(metricManager.getReporters(coreMetricManager.getRegistryName()).isEmpty());
+    deleteCore();
+  }
+
+  @Test
+  public void testRegisterMetrics() {
+    Random random = random();
+
+    String scope = SolrMetricTestUtils.getRandomScope(random);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random);
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random);
+    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+    try {
+      coreMetricManager.registerMetricProducer(scope, producer);
+      assertNotNull(scope);
+      assertNotNull(category);
+      assertNotNull(metrics);
+      assertRegistered(scope, metrics, coreMetricManager);
+    } catch (final IllegalArgumentException e) {
+      assertTrue("expected at least one null but got: scope="+scope+" category="+category+" metrics="+metrics,
+          (scope == null || category == null || metrics == null));
+      assertRegistered(scope, new HashMap<>(), coreMetricManager);
+    }
+  }
+
+  @Test
+  public void testRegisterMetricsWithReplacements() {
+    Random random = random();
+
+    Map<String, Counter> registered = new HashMap<>();
+    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
+
+    int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
+    for (int i = 0; i < iterations; ++i) {
+      Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetricsWithReplacements(random, registered);
+      if (metrics.isEmpty()) {
+        continue;
+      }
+      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+      coreMetricManager.registerMetricProducer(scope, producer);
+      registered.putAll(metrics);
+      assertRegistered(scope, registered, coreMetricManager);
+    }
+  }
+
+  @Test
+  public void testLoadReporter() throws Exception {
+    Random random = random();
+
+    String className = MockMetricReporter.class.getName();
+    String reporterName = TestUtil.randomUnicodeString(random);
+
+    Map<String, Object> attrs = new HashMap<>();
+    attrs.put(FieldType.CLASS_NAME, className);
+    attrs.put(CoreAdminParams.NAME, reporterName);
+
+    boolean shouldDefineConfigurable = random.nextBoolean();
+    String configurable = TestUtil.randomUnicodeString(random);
+    if (shouldDefineConfigurable) attrs.put("configurable", configurable);
+
+    boolean shouldDefinePlugin = random.nextBoolean();
+    PluginInfo pluginInfo = shouldDefinePlugin ? new PluginInfo(TestUtil.randomUnicodeString(random), attrs) : null;
+
+    try {
+      metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+      assertNotNull(pluginInfo);
+      Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+      assertTrue("reporters.size should be > 0, but was + " + reporters.size(), reporters.size() > 0);
+      assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(reporterName));
+      assertTrue("wrong reporter class: " + reporters.get(reporterName), reporters.get(reporterName) instanceof MockMetricReporter);
+    } catch (IllegalArgumentException e) {
+      assertTrue(pluginInfo == null || attrs.get("configurable") == null);
+      assertNull(metricManager.getReporters(coreMetricManager.getRegistryName()).get(reporterName));
+    }
+  }
+
+  private void assertRegistered(String scope, Map<String, Counter> newMetrics, SolrCoreMetricManager coreMetricManager) {
+    if (scope == null) {
+      return;
+    }
+    String filter = "." + scope + ".";
+    MetricRegistry registry = metricManager.registry(coreMetricManager.getRegistryName());
+    assertEquals(newMetrics.size(), registry.getMetrics().
+        keySet().stream().filter(s -> s.contains(filter)).count());
+
+    Map<String, Metric> registeredMetrics = registry.getMetrics().
+        entrySet().stream().filter(e -> e.getKey() != null && e.getKey().contains(filter)).
+        collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
+    for (Map.Entry<String, Metric> entry : registeredMetrics.entrySet()) {
+      String name = entry.getKey();
+      Metric expectedMetric = entry.getValue();
+
+      Metric actualMetric = registry.getMetrics().get(name);
+
+      assertNotNull(actualMetric);
+      assertEquals(expectedMetric, actualMetric);
+    }
+  }
+
+  @Test
+  public void testRegistryName() throws Exception {
+    String collectionName = "my_collection_";
+    String cloudCoreName = "my_collection__shard1_0_replica0";
+    String simpleCoreName = "collection_1_replica0";
+    String simpleRegistryName = "solr.core." + simpleCoreName;
+    String cloudRegistryName = "solr.core." + cloudCoreName;
+    String nestedRegistryName = "solr.core.my_collection_.shard1_0.replica0";
+    // pass through
+    assertEquals(cloudRegistryName, coreMetricManager.createRegistryName(null, cloudCoreName));
+    assertEquals(simpleRegistryName, coreMetricManager.createRegistryName(null, simpleCoreName));
+    // unknown naming scheme -> pass through
+    assertEquals(simpleRegistryName, coreMetricManager.createRegistryName(collectionName, simpleCoreName));
+    // cloud collection
+    assertEquals(nestedRegistryName, coreMetricManager.createRegistryName(collectionName, cloudCoreName));
+
+  }
+}