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/20 09:12:38 UTC

[2/3] lucene-solr:master: Squashed commit of branch 'feature/metrics', containing: SOLR-4735: Improve Solr metrics reporting SOLR-9812: Implement /admin/metrics API SOLR-9805: Use metrics-jvm library to instrument jvm internals SOLR-9788:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/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/8bbdb624/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/8bbdb624/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/8bbdb624/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/8bbdb624/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/8bbdb624/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..dbc4b35 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,12 @@ 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.FileDescriptorRatioGauge;
+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 +66,10 @@ 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.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, "bufferPools");
+      metricManager.registerAll(registry, new ClassLoadingGaugeSet(), true, "classLoading");
+      metricManager.register(registry, new FileDescriptorRatioGauge(), true, "fileDescriptorRatio");
+      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/8bbdb624/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/8bbdb624/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/8bbdb624/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/8bbdb624/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/8bbdb624/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/8bbdb624/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));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
new file mode 100644
index 0000000..ecddfba
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricManagerTest.java
@@ -0,0 +1,273 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+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.util.NamedList;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.junit.Test;
+
+public class SolrMetricManagerTest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testOverridableRegistryName() throws Exception {
+    Random r = random();
+    String originalName = TestUtil.randomSimpleString(r, 1, 10);
+    String targetName = TestUtil.randomSimpleString(r, 1, 10);
+    // no override
+    String result = SolrMetricManager.overridableRegistryName(originalName);
+    assertEquals(SolrMetricManager.REGISTRY_NAME_PREFIX + originalName, result);
+    // with override
+    System.setProperty(SolrMetricManager.REGISTRY_NAME_PREFIX + originalName, targetName);
+    result = SolrMetricManager.overridableRegistryName(originalName);
+    assertEquals(SolrMetricManager.REGISTRY_NAME_PREFIX + targetName, result);
+  }
+
+  @Test
+  public void testMoveMetrics() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    Map<String, Counter> metrics1 = SolrMetricTestUtils.getRandomMetrics(r, true);
+    Map<String, Counter> metrics2 = SolrMetricTestUtils.getRandomMetrics(r, true);
+    String fromName = TestUtil.randomSimpleString(r, 1, 10);
+    String toName = TestUtil.randomSimpleString(r, 1, 10);
+    // register test metrics
+    for (Map.Entry<String, Counter> entry : metrics1.entrySet()) {
+      metricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics1");
+    }
+    for (Map.Entry<String, Counter> entry : metrics2.entrySet()) {
+      metricManager.register(fromName, entry.getValue(), false, entry.getKey(), "metrics2");
+    }
+    assertEquals(metrics1.size() + metrics2.size(), metricManager.registry(fromName).getMetrics().size());
+
+    // move metrics1
+    metricManager.moveMetrics(fromName, toName, new SolrMetricManager.PrefixFilter("metrics1"));
+    // check the remaining metrics
+    Map<String, Metric> fromMetrics = metricManager.registry(fromName).getMetrics();
+    assertEquals(metrics2.size(), fromMetrics.size());
+    for (Map.Entry<String, Counter> entry : metrics2.entrySet()) {
+      Object value = fromMetrics.get(SolrMetricManager.mkName(entry.getKey(), "metrics2"));
+      assertNotNull(value);
+      assertEquals(entry.getValue(), value);
+    }
+    // check the moved metrics
+    Map<String, Metric> toMetrics = metricManager.registry(toName).getMetrics();
+    assertEquals(metrics1.size(), toMetrics.size());
+    for (Map.Entry<String, Counter> entry : metrics1.entrySet()) {
+      Object value = toMetrics.get(SolrMetricManager.mkName(entry.getKey(), "metrics1"));
+      assertNotNull(value);
+      assertEquals(entry.getValue(), value);
+    }
+
+    // move all remaining metrics
+    metricManager.moveMetrics(fromName, toName, null);
+    fromMetrics = metricManager.registry(fromName).getMetrics();
+    assertEquals(0, fromMetrics.size());
+    toMetrics = metricManager.registry(toName).getMetrics();
+    assertEquals(metrics1.size() + metrics2.size(), toMetrics.size());
+  }
+
+  @Test
+  public void testRegisterAll() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(r, true);
+    MetricRegistry mr = new MetricRegistry();
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      mr.register(entry.getKey(), entry.getValue());
+    }
+
+    String registryName = TestUtil.randomSimpleString(r, 1, 10);
+    assertEquals(0, metricManager.registry(registryName).getMetrics().size());
+    metricManager.registerAll(registryName, mr, false);
+    // this should simply skip existing names
+    metricManager.registerAll(registryName, mr, true);
+    // this should produce error
+    try {
+      metricManager.registerAll(registryName, mr, false);
+      fail("registerAll with duplicate metric names should fail");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testClearMetrics() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(r, true);
+    String registryName = TestUtil.randomSimpleString(r, 1, 10);
+
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "bar");
+    }
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo", "baz");
+    }
+    for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+      metricManager.register(registryName, entry.getValue(), false, entry.getKey(), "foo");
+    }
+
+    assertEquals(metrics.size() * 3, metricManager.registry(registryName).getMetrics().size());
+
+    // clear "foo.bar"
+    Set<String> removed = metricManager.clearMetrics(registryName, "foo", "bar");
+    assertEquals(metrics.size(), removed.size());
+    for (String s : removed) {
+      assertTrue(s.startsWith("foo.bar."));
+    }
+    removed = metricManager.clearMetrics(registryName, "foo", "baz");
+    assertEquals(metrics.size(), removed.size());
+    for (String s : removed) {
+      assertTrue(s.startsWith("foo.baz."));
+    }
+    // perhaps surprisingly, this works too - see PrefixFilter docs
+    removed = metricManager.clearMetrics(registryName, "fo");
+    assertEquals(metrics.size(), removed.size());
+    for (String s : removed) {
+      assertTrue(s.startsWith("foo."));
+    }
+  }
+
+  @Test
+  public void testSimpleMetrics() throws Exception {
+    Random r = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    String registryName = TestUtil.randomSimpleString(r, 1, 10);
+
+    metricManager.counter(registryName, "simple_counter", "foo", "bar");
+    metricManager.timer(registryName, "simple_timer", "foo", "bar");
+    metricManager.meter(registryName, "simple_meter", "foo", "bar");
+    metricManager.histogram(registryName, "simple_histogram", "foo", "bar");
+    Map<String, Metric> metrics = metricManager.registry(registryName).getMetrics();
+    assertEquals(4, metrics.size());
+    for (Map.Entry<String, Metric> entry : metrics.entrySet()) {
+      assertTrue(entry.getKey().startsWith("foo.bar.simple_"));
+    }
+  }
+
+  @Test
+  public void testRegistryName() throws Exception {
+    Random r = random();
+
+    String name = TestUtil.randomSimpleString(r, 1, 10);
+
+    String result = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name, "collection1");
+    assertEquals("solr.core." + name + ".collection1", result);
+    // try it with already prefixed name - group will be ignored
+    result = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, result);
+    assertEquals("solr.core." + name + ".collection1", result);
+    // try it with already prefixed name but with additional segments
+    result = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, result, "shard1", "replica1");
+    assertEquals("solr.core." + name + ".collection1.shard1.replica1", result);
+  }
+
+  @Test
+  public void testReporters() throws Exception {
+    Random r = random();
+
+    SolrResourceLoader loader = new SolrResourceLoader();
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    PluginInfo[] plugins = new PluginInfo[] {
+        createPluginInfo("universal_foo", null, null),
+        createPluginInfo("multigroup_foo", "jvm, node, core", null),
+        createPluginInfo("multiregistry_foo", null, "solr.node, solr.core.collection1"),
+        createPluginInfo("specific_foo", null, "solr.core.collection1"),
+        createPluginInfo("node_foo", "node", null),
+        createPluginInfo("core_foo", "core", null)
+    };
+
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.node);
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    assertEquals(4, reporters.size());
+    assertTrue(reporters.containsKey("universal_foo"));
+    assertTrue(reporters.containsKey("multigroup_foo"));
+    assertTrue(reporters.containsKey("node_foo"));
+    assertTrue(reporters.containsKey("multiregistry_foo"));
+
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.core, "collection1");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, "collection1"));
+    assertEquals(5, reporters.size());
+    assertTrue(reporters.containsKey("universal_foo"));
+    assertTrue(reporters.containsKey("multigroup_foo"));
+    assertTrue(reporters.containsKey("specific_foo"));
+    assertTrue(reporters.containsKey("core_foo"));
+    assertTrue(reporters.containsKey("multiregistry_foo"));
+
+    metricManager.loadReporters(plugins, loader, SolrInfoMBean.Group.jvm);
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm));
+    assertEquals(2, reporters.size());
+    assertTrue(reporters.containsKey("universal_foo"));
+    assertTrue(reporters.containsKey("multigroup_foo"));
+
+    metricManager.removeRegistry("solr.jvm");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.jvm));
+    assertEquals(0, reporters.size());
+
+    metricManager.removeRegistry("solr.node");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    assertEquals(0, reporters.size());
+
+    metricManager.removeRegistry("solr.core.collection1");
+    reporters = metricManager.getReporters(
+        SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, "collection1"));
+    assertEquals(0, reporters.size());
+
+  }
+
+  private PluginInfo createPluginInfo(String name, String group, String registry) {
+    Map<String,String> attrs = new HashMap<>();
+    attrs.put("name", name);
+    attrs.put("class", MockMetricReporter.class.getName());
+    if (group != null) {
+      attrs.put("group", group);
+    }
+    if (registry != null) {
+      attrs.put("registry", registry);
+    }
+    NamedList initArgs = new NamedList();
+    initArgs.add("configurable", "true");
+    return new PluginInfo("SolrMetricReporter", attrs, initArgs, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
new file mode 100644
index 0000000..b275919
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricReporterTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.apache.solr.schema.FieldType;
+import org.junit.Test;
+
+public class SolrMetricReporterTest extends LuceneTestCase {
+
+  @Test
+  public void testInit() throws Exception {
+    Random random = random();
+
+    SolrMetricManager metricManager = new SolrMetricManager();
+
+    final String registryName = TestUtil.randomSimpleString(random);
+    final MockMetricReporter reporter = new MockMetricReporter(metricManager, registryName);
+
+    Map<String, Object> attrs = new HashMap<>();
+    attrs.put(FieldType.CLASS_NAME, MockMetricReporter.class.getName());
+    attrs.put(CoreAdminParams.NAME, TestUtil.randomUnicodeString(random));
+
+    boolean shouldDefineConfigurable = random.nextBoolean();
+    String configurable = TestUtil.randomUnicodeString(random);
+    if (shouldDefineConfigurable) attrs.put("configurable", configurable);
+
+    boolean shouldDefinePlugin = random.nextBoolean();
+    String type = TestUtil.randomUnicodeString(random);
+    PluginInfo pluginInfo = shouldDefinePlugin ? new PluginInfo(type, attrs) : null;
+
+    try {
+      reporter.init(pluginInfo);
+      assertNotNull(pluginInfo);
+      assertEquals(configurable, attrs.get("configurable"));
+      assertTrue(reporter.didValidate);
+      assertNotNull(reporter.configurable);
+      assertEquals(configurable, reporter.configurable);
+    } catch (IllegalStateException e) {
+      assertTrue(pluginInfo == null || attrs.get("configurable") == null);
+      assertTrue(reporter.didValidate);
+      assertNull(reporter.configurable);
+    } finally {
+      reporter.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
new file mode 100644
index 0000000..44ae9db
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricTestUtils.java
@@ -0,0 +1,140 @@
+/*
+ * 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.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import com.codahale.metrics.Counter;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrInfoMBean;
+
+public final class SolrMetricTestUtils {
+
+  private static final int                    MAX_ITERATIONS = 100;
+  private static final SolrInfoMBean.Category CATEGORIES[]   = SolrInfoMBean.Category.values();
+
+  public static String getRandomScope(Random random) {
+    return getRandomScope(random, random.nextBoolean());
+  }
+
+  public static String getRandomScope(Random random, boolean shouldDefineScope) {
+    return shouldDefineScope ? TestUtil.randomSimpleString(random, 1, 10) : null; // must be simple string for JMX publishing
+  }
+
+  public static SolrInfoMBean.Category getRandomCategory(Random random) {
+    return getRandomCategory(random, random.nextBoolean());
+  }
+
+  public static SolrInfoMBean.Category getRandomCategory(Random random, boolean shouldDefineCategory) {
+    return shouldDefineCategory ? CATEGORIES[TestUtil.nextInt(random, 0, CATEGORIES.length - 1)] : null;
+  }
+
+  public static Map<String, Counter> getRandomMetrics(Random random) {
+    return getRandomMetrics(random, random.nextBoolean());
+  }
+
+  public static Map<String, Counter> getRandomMetrics(Random random, boolean shouldDefineMetrics) {
+    return shouldDefineMetrics ? getRandomMetricsWithReplacements(random, new HashMap<>()) : null;
+  }
+
+  public static final String SUFFIX = "_testing";
+
+  public static Map<String, Counter> getRandomMetricsWithReplacements(Random random, Map<String, Counter> existing) {
+    HashMap<String, Counter> metrics = new HashMap<>();
+    ArrayList<String> existingKeys = new ArrayList<>(existing.keySet());
+
+    int numMetrics = TestUtil.nextInt(random, 1, MAX_ITERATIONS);
+    for (int i = 0; i < numMetrics; ++i) {
+      boolean shouldReplaceMetric = !existing.isEmpty() && random.nextBoolean();
+      String name = shouldReplaceMetric
+          ? existingKeys.get(TestUtil.nextInt(random, 0, existingKeys.size() - 1))
+          : TestUtil.randomSimpleString(random, 1, 10) + SUFFIX; // must be simple string for JMX publishing
+
+      Counter counter = new Counter();
+      counter.inc(random.nextLong());
+      metrics.put(name, counter);
+    }
+
+    return metrics;
+  }
+
+  public static SolrMetricProducer getProducerOf(SolrMetricManager metricManager, SolrInfoMBean.Category category, String scope, Map<String, Counter> metrics) {
+    return new SolrMetricProducer() {
+      @Override
+      public Collection<String> initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+        if (metrics == null || metrics.isEmpty()) {
+          return Collections.emptyList();
+        }
+        for (Map.Entry<String, Counter> entry : metrics.entrySet()) {
+          manager.counter(registry, entry.getKey(), category.toString(), scope);
+        }
+        return metrics.keySet();
+      }
+
+      @Override
+      public String getName() {
+        return scope;
+      }
+
+      @Override
+      public String getVersion() {
+        return "0.0";
+      }
+
+      @Override
+      public String getDescription() {
+        return "foo";
+      }
+
+      @Override
+      public Category getCategory() {
+        return category;
+      }
+
+      @Override
+      public String getSource() {
+        return null;
+      }
+
+      @Override
+      public URL[] getDocs() {
+        return new URL[0];
+      }
+
+      @Override
+      public NamedList getStatistics() {
+        return null;
+      }
+
+      @Override
+      public String toString() {
+        return "SolrMetricProducer.of{" +
+            "\ncategory=" + category +
+            "\nscope=" + scope +
+            "\nmetrics=" + metrics +
+            "\n}";
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
new file mode 100644
index 0000000..c6449ac
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/SolrMetricsIntegrationTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Random;
+
+import com.codahale.metrics.Timer;
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.metrics.reporters.MockMetricReporter;
+import org.apache.solr.util.TestHarness;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SolrMetricsIntegrationTest extends SolrTestCaseJ4 {
+  private static final int MAX_ITERATIONS = 20;
+  private static final String CORE_NAME = "metrics_integration";
+  private static final String METRIC_NAME = "requestTimes";
+  private static final String HANDLER_NAME = "standard";
+  private static final String[] REPORTER_NAMES = {"reporter1", "reporter2"};
+  private static final String UNIVERSAL = "universal";
+  private static final String SPECIFIC = "specific";
+  private static final String MULTIGROUP = "multigroup";
+  private static final String MULTIREGISTRY = "multiregistry";
+  private static final String[] INITIAL_REPORTERS = {REPORTER_NAMES[0], REPORTER_NAMES[1], UNIVERSAL, SPECIFIC, MULTIGROUP, MULTIREGISTRY};
+  private static final String[] RENAMED_REPORTERS = {REPORTER_NAMES[0], REPORTER_NAMES[1], UNIVERSAL, MULTIGROUP};
+  private static final SolrInfoMBean.Category HANDLER_CATEGORY = SolrInfoMBean.Category.QUERYHANDLER;
+
+  private CoreContainer cc;
+  private SolrMetricManager metricManager;
+
+  @Before
+  public void beforeTest() throws Exception {
+    Path home = Paths.get(TEST_HOME());
+    // define these properties, they are used in solrconfig.xml
+    System.setProperty("solr.test.sys.prop1", "propone");
+    System.setProperty("solr.test.sys.prop2", "proptwo");
+    String solrXml = FileUtils.readFileToString(Paths.get(home.toString(), "solr-metricreporter.xml").toFile(), "UTF-8");
+    NodeConfig cfg = SolrXmlConfig.fromString(new SolrResourceLoader(home), solrXml);
+    cc = createCoreContainer(cfg,
+        new TestHarness.TestCoresLocator(DEFAULT_TEST_CORENAME, initCoreDataDir.getAbsolutePath(), "solrconfig.xml", "schema.xml"));
+    h.coreName = DEFAULT_TEST_CORENAME;
+    metricManager = cc.getMetricManager();
+    // initially there are more reporters, because two of them are added via a matching collection name
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters("solr.core." + DEFAULT_TEST_CORENAME);
+    assertEquals(INITIAL_REPORTERS.length, reporters.size());
+    assertTrue(reporters.keySet().containsAll(Arrays.asList(INITIAL_REPORTERS)));
+    // test rename operation
+    cc.rename(DEFAULT_TEST_CORENAME, CORE_NAME);
+    h.coreName = CORE_NAME;
+    cfg = cc.getConfig();
+    PluginInfo[] plugins = cfg.getMetricReporterPlugins();
+    assertNotNull(plugins);
+    assertEquals(10, plugins.length);
+    reporters = metricManager.getReporters("solr.node");
+    assertEquals(4, reporters.size());
+    assertTrue("Reporter '" + REPORTER_NAMES[0] + "' missing in solr.node", reporters.containsKey(REPORTER_NAMES[0]));
+    assertTrue("Reporter '" + UNIVERSAL + "' missing in solr.node", reporters.containsKey(UNIVERSAL));
+    assertTrue("Reporter '" + MULTIGROUP + "' missing in solr.node", reporters.containsKey(MULTIGROUP));
+    assertTrue("Reporter '" + MULTIREGISTRY + "' missing in solr.node", reporters.containsKey(MULTIREGISTRY));
+    SolrMetricReporter reporter = reporters.get(REPORTER_NAMES[0]);
+    assertTrue("Reporter " + reporter + " is not an instance of " + MockMetricReporter.class.getName(),
+        reporter instanceof  MockMetricReporter);
+    reporter = reporters.get(UNIVERSAL);
+    assertTrue("Reporter " + reporter + " is not an instance of " + MockMetricReporter.class.getName(),
+        reporter instanceof  MockMetricReporter);
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    SolrCoreMetricManager coreMetricManager = h.getCore().getCoreMetricManager();
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+
+    deleteCore();
+
+    for (String reporterName : RENAMED_REPORTERS) {
+      SolrMetricReporter reporter = reporters.get(reporterName);
+      MockMetricReporter mockReporter = (MockMetricReporter) reporter;
+      assertTrue("Reporter " + reporterName + " was not closed: " + mockReporter, mockReporter.didClose);
+    }
+  }
+
+  @Test
+  public void testConfigureReporter() throws Exception {
+    Random random = random();
+
+    String metricName = SolrMetricManager.mkName(METRIC_NAME, HANDLER_CATEGORY.toString(), HANDLER_NAME);
+    SolrCoreMetricManager coreMetricManager = h.getCore().getCoreMetricManager();
+    Timer timer = (Timer) metricManager.timer(coreMetricManager.getRegistryName(), metricName);
+
+    long initialCount = timer.getCount();
+
+    int iterations = TestUtil.nextInt(random, 0, MAX_ITERATIONS);
+    for (int i = 0; i < iterations; ++i) {
+      h.query(req("*"));
+    }
+
+    long finalCount = timer.getCount();
+    assertEquals("metric counter incorrect", iterations, finalCount - initialCount);
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+    assertEquals(RENAMED_REPORTERS.length, reporters.size());
+
+    // SPECIFIC and MULTIREGISTRY were skipped because they were
+    // specific to collection1
+    for (String reporterName : RENAMED_REPORTERS) {
+      SolrMetricReporter reporter = reporters.get(reporterName);
+      assertNotNull("Reporter " + reporterName + " was not found.", reporter);
+      assertTrue(reporter instanceof MockMetricReporter);
+
+      MockMetricReporter mockReporter = (MockMetricReporter) reporter;
+      assertTrue("Reporter " + reporterName + " was not initialized: " + mockReporter, mockReporter.didInit);
+      assertTrue("Reporter " + reporterName + " was not validated: " + mockReporter, mockReporter.didValidate);
+      assertFalse("Reporter " + reporterName + " was incorrectly closed: " + mockReporter, mockReporter.didClose);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java b/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
new file mode 100644
index 0000000..2ecc33b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/MockMetricReporter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.IOException;
+import java.util.Locale;
+import java.util.NoSuchElementException;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricReporter;
+
+public class MockMetricReporter extends SolrMetricReporter {
+
+  public String configurable;
+
+  public boolean didInit = false;
+  public boolean didClose = false;
+  public boolean didValidate = false;
+
+  public MockMetricReporter(SolrMetricManager metricManager, String registryName) {
+    super(metricManager, registryName);
+  }
+
+  @Override
+  public void init(PluginInfo pluginInfo) {
+    super.init(pluginInfo);
+    didInit = true;
+  }
+
+  @Override
+  public void close() throws IOException {
+    didClose = true;
+  }
+
+  @Override
+  protected void validate() throws IllegalStateException {
+    didValidate = true;
+    if (configurable == null) {
+      throw new IllegalStateException("MockMetricReporter::configurable not defined.");
+    }
+  }
+
+  public void setConfigurable(String configurable) {
+    this.configurable = configurable;
+  }
+
+  public Metric reportMetric(String metricName) throws NoSuchElementException {
+    MetricRegistry registry = metricManager.registry(registryName);
+    Metric metric = registry.getMetrics().get(metricName);
+    if (metric == null) {
+      throw new NoSuchElementException("Metric was not found for metric name = " + metricName);
+    }
+
+    return metric;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(Locale.ENGLISH, "[%s@%s: configurable = %s, didInit = %b, didValidate = %b, didClose = %b]",
+        getClass().getName(), Integer.toHexString(hashCode()), configurable, didInit, didValidate, didClose);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
new file mode 100644
index 0000000..ea452b2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrJmxReporterTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.ObjectInstance;
+import javax.management.ObjectName;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import com.codahale.metrics.Counter;
+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.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricReporter;
+import org.apache.solr.metrics.SolrMetricTestUtils;
+import org.apache.solr.schema.FieldType;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SolrJmxReporterTest extends SolrTestCaseJ4 {
+
+  private static final int MAX_ITERATIONS = 20;
+
+  private String domain;
+
+  private SolrCoreMetricManager coreMetricManager;
+  private SolrMetricManager metricManager;
+  private SolrJmxReporter reporter;
+  private MBeanServer mBeanServer;
+  private String reporterName;
+
+  @Before
+  public void beforeTest() throws Exception {
+    initCore("solrconfig-basic.xml", "schema.xml");
+
+    final SolrCore core = h.getCore();
+    domain = core.getName();
+
+    coreMetricManager = core.getCoreMetricManager();
+    metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
+    PluginInfo pluginInfo = createReporterPluginInfo();
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+
+    Map<String, SolrMetricReporter> reporters = metricManager.getReporters(coreMetricManager.getRegistryName());
+    assertTrue("reporters.size should be > 0, but was + " + reporters.size(), reporters.size() > 0);
+    reporterName = pluginInfo.name;
+    assertNotNull("reporter " + reporterName + " not present among " + reporters, reporters.get(reporterName));
+    assertTrue("wrong reporter class: " + reporters.get(reporterName), reporters.get(reporterName) instanceof SolrJmxReporter);
+
+    reporter = (SolrJmxReporter) reporters.get(reporterName);
+    mBeanServer = reporter.getMBeanServer();
+    assertNotNull("MBean server not found.", mBeanServer);
+  }
+
+  private PluginInfo createReporterPluginInfo() {
+    Random random = random();
+    String className = SolrJmxReporter.class.getName();
+    String reporterName = TestUtil.randomSimpleString(random, 1, 10);
+
+    Map<String, Object> attrs = new HashMap<>();
+    attrs.put(FieldType.CLASS_NAME, className);
+    attrs.put(CoreAdminParams.NAME, reporterName);
+
+    boolean shouldOverrideDomain = random.nextBoolean();
+    if (shouldOverrideDomain) {
+      domain = TestUtil.randomSimpleString(random);
+      attrs.put("domain", domain);
+    }
+
+    return new PluginInfo(TestUtil.randomUnicodeString(random), attrs);
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    metricManager.closeReporters(coreMetricManager.getRegistryName());
+    Set<ObjectInstance> objects =
+        mBeanServer.queryMBeans(ObjectName.getInstance(domain + ":*"), null);
+    assertTrue(objects.isEmpty());
+
+    coreMetricManager.close();
+    deleteCore();
+  }
+
+  @Test
+  public void testReportMetrics() throws Exception {
+    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);
+      SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+      coreMetricManager.registerMetricProducer(scope, producer);
+      registered.putAll(metrics);
+      //waitForListener();
+      Set<ObjectInstance> objects = mBeanServer.queryMBeans(null, null);
+      assertEquals(registered.size(), objects.stream().
+          filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
+                      reporterName.equals(o.getObjectName().getKeyProperty("reporter"))).count());
+    }
+  }
+
+  @Test
+  public void testReloadCore() throws Exception {
+    Random random = random();
+
+    String scope = SolrMetricTestUtils.getRandomScope(random, true);
+    SolrInfoMBean.Category category = SolrMetricTestUtils.getRandomCategory(random, true);
+    Map<String, Counter> metrics = SolrMetricTestUtils.getRandomMetrics(random, true);
+    SolrMetricProducer producer = SolrMetricTestUtils.getProducerOf(metricManager, category, scope, metrics);
+    coreMetricManager.registerMetricProducer(scope, producer);
+    Set<ObjectInstance> objects = mBeanServer.queryMBeans(null, null);
+    assertEquals(metrics.size(), objects.stream().
+        filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
+            reporterName.equals(o.getObjectName().getKeyProperty("reporter"))).count());
+
+    h.getCoreContainer().reload(h.getCore().getName());
+    PluginInfo pluginInfo = createReporterPluginInfo();
+    metricManager.loadReporter(coreMetricManager.getRegistryName(), coreMetricManager.getCore().getResourceLoader(), pluginInfo);
+    coreMetricManager.registerMetricProducer(scope, producer);
+
+    objects = mBeanServer.queryMBeans(null, null);
+    assertEquals(metrics.size(), objects.stream().
+        filter(o -> scope.equals(o.getObjectName().getKeyProperty("scope")) &&
+            pluginInfo.name.equals(o.getObjectName().getKeyProperty("reporter"))).count());
+  }
+
+}