You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2016/12/20 16:39:55 UTC

[19/24] lucene-solr:jira/solr-8593: 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

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: Use instrumented jetty classes


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8bbdb624
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8bbdb624
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8bbdb624

Branch: refs/heads/jira/solr-8593
Commit: 8bbdb6248c5de3f3bd61501ba42a50aeec29c78b
Parents: 84bbb8f
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Dec 20 09:31:24 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Dec 20 09:31:24 2016 +0100

----------------------------------------------------------------------
 .../idea/solr/contrib/analytics/analytics.iml   |   1 +
 lucene/ivy-versions.properties                  |   2 +
 solr/CHANGES.txt                                |  13 +
 .../plugin/AnalyticsStatisticsCollector.java    |   4 +-
 solr/core/ivy.xml                               |   1 -
 .../apache/solr/cloud/OverseerStatusCmd.java    |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |  66 +-
 .../java/org/apache/solr/core/NodeConfig.java   |  18 +-
 .../java/org/apache/solr/core/PluginInfo.java   |   4 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  73 +++
 .../org/apache/solr/core/SolrInfoMBean.java     |  10 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |  14 +-
 .../apache/solr/handler/RequestHandlerBase.java |  58 +-
 .../solr/handler/admin/MetricsHandler.java      | 164 +++++
 .../solr/metrics/SolrCoreMetricManager.java     | 154 +++++
 .../org/apache/solr/metrics/SolrMetricInfo.java | 104 +++
 .../apache/solr/metrics/SolrMetricManager.java  | 652 +++++++++++++++++++
 .../apache/solr/metrics/SolrMetricProducer.java |  38 ++
 .../apache/solr/metrics/SolrMetricReporter.java |  83 +++
 .../org/apache/solr/metrics/package-info.java   |  23 +
 .../solr/metrics/reporters/SolrJmxReporter.java | 284 ++++++++
 .../solr/metrics/reporters/package-info.java    |  22 +
 .../solr/security/PermissionNameProvider.java   |   1 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  27 +
 .../src/java/org/apache/solr/util/JmxUtil.java  |  78 +++
 .../org/apache/solr/util/stats/MetricUtils.java | 144 ++++
 .../org/apache/solr/util/stats/TimerUtils.java  |  58 --
 .../src/test-files/solr/solr-metricreporter.xml |  57 ++
 .../solr/handler/admin/MetricsHandlerTest.java  |  97 +++
 .../solr/metrics/SolrCoreMetricManagerTest.java | 172 +++++
 .../solr/metrics/SolrMetricManagerTest.java     | 273 ++++++++
 .../solr/metrics/SolrMetricReporterTest.java    |  69 ++
 .../solr/metrics/SolrMetricTestUtils.java       | 140 ++++
 .../metrics/SolrMetricsIntegrationTest.java     | 143 ++++
 .../metrics/reporters/MockMetricReporter.java   |  80 +++
 .../metrics/reporters/SolrJmxReporterTest.java  | 156 +++++
 .../apache/solr/util/stats/MetricUtilsTest.java |  58 ++
 .../apache/solr/util/stats/TimerUtilsTest.java  |  58 --
 solr/licenses/metrics-jetty-LICENSE-ASL.txt     | 203 ++++++
 solr/licenses/metrics-jetty-NOTICE.txt          |  12 +
 solr/licenses/metrics-jetty9-3.1.2.jar.sha1     |   1 +
 solr/licenses/metrics-json-LICENSE-ASL.txt      | 203 ++++++
 solr/licenses/metrics-json-NOTICE.txt           |  12 +
 solr/licenses/metrics-jvm-3.1.2.jar.sha1        |   1 +
 solr/licenses/metrics-jvm-LICENSE-ASL.txt       | 203 ++++++
 solr/licenses/metrics-jvm-NOTICE.txt            |  12 +
 solr/licenses/metrics-servlets-LICENSE-ASL.txt  | 203 ++++++
 solr/licenses/metrics-servlets-NOTICE.txt       |  12 +
 solr/server/build.xml                           |   4 +-
 solr/server/etc/jetty.xml                       |  20 +-
 solr/server/ivy.xml                             |  11 +-
 .../apache/solr/common/params/CommonParams.java |   4 +-
 .../java/org/apache/solr/util/TestHarness.java  |   9 +
 53 files changed, 4145 insertions(+), 168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/dev-tools/idea/solr/contrib/analytics/analytics.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/solr/contrib/analytics/analytics.iml b/dev-tools/idea/solr/contrib/analytics/analytics.iml
index 10f51a7..d63d3e2 100644
--- a/dev-tools/idea/solr/contrib/analytics/analytics.iml
+++ b/dev-tools/idea/solr/contrib/analytics/analytics.iml
@@ -14,6 +14,7 @@
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="library" name="Solr core library" level="project" />
     <orderEntry type="library" name="Solrj library" level="project" />
+    <orderEntry type="library" name="Solr example library" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
     <orderEntry type="module" scope="TEST" module-name="solr-test-framework" />
     <orderEntry type="module" module-name="lucene-core" />

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index ffc54a8..2f44f7e 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -76,6 +76,8 @@ com.sun.jersey.version = 1.9
 io.dropwizard.metrics.version = 3.1.2
 /io.dropwizard.metrics/metrics-core = ${io.dropwizard.metrics.version}
 /io.dropwizard.metrics/metrics-healthchecks = ${io.dropwizard.metrics.version}
+/io.dropwizard.metrics/metrics-jetty9 = ${io.dropwizard.metrics.version}
+/io.dropwizard.metrics/metrics-jvm = ${io.dropwizard.metrics.version}
 
 io.netty.netty-all.version = 4.0.36.Final
 /io.netty/netty-all = ${io.netty.netty-all.version}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d04d491..519cdfa 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -173,11 +173,24 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
+<<<<<<< HEAD
 * SOLR-9513: Generic authentication plugins (GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin) that delegate
   all functionality to Hadoop authentication framework. (Hrishikesh Gadre via Ishan Chattopadhyaya)
 
 * SOLR-9860: Enable configuring invariantParams via HttpSolrClient.Builder (Hrishikesh Gadre, Ishan Chattopadhyaya)
  
+* SOLR-4735: Improve metrics reporting. This uses the dropwizard metric library, adding an internal API
+  for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
+  reporter have been added (Alan Woodward, Jeff Wartes, Christine Poerschke, Kelvin Wong, shalin, ab)
+
+* SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. (shalin)
+
+* SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)
+
+* SOLR-9812: Added a new /admin/metrics API to return all metrics collected by Solr via API. API supports two
+  optional parameters 'group' (all,jvm,jetty,http,node,core) and 'type' (all,counter,timer,gauge,histogram) both
+  of which are multi-valued. Example: http://localhost:8983/solr/admin/metrics?group=jvm,jetty&type=counter
+  (shalin)
 
 Optimizations
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
index e64c950..b22dcb5 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/plugin/AnalyticsStatisticsCollector.java
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.codahale.metrics.Timer;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 
 public class AnalyticsStatisticsCollector {
   private final AtomicLong numRequests;
@@ -95,7 +95,7 @@ public class AnalyticsStatisticsCollector {
     lst.add("rangeFacets", numRangeFacets.longValue());
     lst.add("queryFacets", numQueryFacets.longValue());
     lst.add("queriesInQueryFacets", numQueries.longValue());
-    TimerUtils.addMetrics(lst, requestTimes);
+    MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 5710da9..67e4379 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -50,7 +50,6 @@
     <dependency org="log4j" name="log4j" rev="${/log4j/log4j}" conf="compile"/>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="compile"/>
     <dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="compile"/>
-    <dependency org="io.dropwizard.metrics" name="metrics-core" rev="${/io.dropwizard.metrics/metrics-core}" conf="compile" />
 
     <dependency org="org.easymock" name="easymock" rev="${/org.easymock/easymock}" conf="test"/>
     <dependency org="cglib" name="cglib-nodep" rev="${/cglib/cglib-nodep}" conf="test"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java b/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
index 901a884..a24deb1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerStatusCmd.java
@@ -30,7 +30,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -100,7 +100,7 @@ public class OverseerStatusCmd implements Cmd {
         lst.add("errors", errors);
       }
       Timer timer = entry.getValue().requestTime;
-      TimerUtils.addMetrics(lst, timer);
+      MetricUtils.addMetrics(lst, timer);
     }
     results.add("overseer_operations", overseerStats);
     results.add("collection_operations", collectionStats);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 7c38b81..6e640bc 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -33,6 +33,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
+import com.codahale.metrics.Gauge;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -58,6 +59,7 @@ import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.admin.MetricsHandler;
 import org.apache.solr.handler.admin.SecurityConfHandler;
 import org.apache.solr.handler.admin.SecurityConfHandlerLocal;
 import org.apache.solr.handler.admin.SecurityConfHandlerZk;
@@ -65,6 +67,8 @@ import org.apache.solr.handler.admin.ZookeeperInfoHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationPlugin;
@@ -85,6 +89,7 @@ import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PAT
 import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.INFO_HANDLER_PATH;
+import static org.apache.solr.common.params.CommonParams.METRICS_PATH;
 import static org.apache.solr.common.params.CommonParams.ZK_PATH;
 import static org.apache.solr.security.AuthenticationPlugin.AUTHENTICATION_PLUGIN_PROP;
 
@@ -156,6 +161,10 @@ public class CoreContainer {
 
   private BackupRepositoryFactory backupRepoFactory;
 
+  protected SolrMetricManager metricManager;
+
+  protected MetricsHandler metricsHandler;
+
   /**
    * This method instantiates a new instance of {@linkplain BackupRepository}.
    *
@@ -423,6 +432,10 @@ public class CoreContainer {
     return pkiAuthenticationPlugin;
   }
 
+  public SolrMetricManager getMetricManager() {
+    return metricManager;
+  }
+
   //-------------------------------------------------------------------
   // Initialization / Cleanup
   //-------------------------------------------------------------------
@@ -463,28 +476,45 @@ public class CoreContainer {
 
     MDCLoggingContext.setNode(this);
 
+    metricManager = new SolrMetricManager();
+
     securityConfHandler = isZooKeeperAware() ? new SecurityConfHandlerZk(this) : new SecurityConfHandlerLocal(this);
     reloadSecurityProperties();
     this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
 
-    containerHandlers.put(ZK_PATH, new ZookeeperInfoHandler(this));
-    collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
-    containerHandlers.put(COLLECTIONS_HANDLER_PATH, collectionsHandler);
-    infoHandler        = createHandler(cfg.getInfoHandlerClass(), InfoHandler.class);
-    containerHandlers.put(INFO_HANDLER_PATH, infoHandler);
-    coreAdminHandler   = createHandler(cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
-    containerHandlers.put(CORES_HANDLER_PATH, coreAdminHandler);
-    configSetsHandler = createHandler(cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
-    containerHandlers.put(CONFIGSETS_HANDLER_PATH, configSetsHandler);
+    createHandler(ZK_PATH, ZookeeperInfoHandler.class.getName(), ZookeeperInfoHandler.class);
+    collectionsHandler = createHandler(COLLECTIONS_HANDLER_PATH, cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
+    infoHandler        = createHandler(INFO_HANDLER_PATH, cfg.getInfoHandlerClass(), InfoHandler.class);
+    coreAdminHandler   = createHandler(CORES_HANDLER_PATH, cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
+    configSetsHandler = createHandler(CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
+    metricsHandler = createHandler(METRICS_PATH, MetricsHandler.class.getName(), MetricsHandler.class);
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
+    securityConfHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
     if(pkiAuthenticationPlugin != null)
       containerHandlers.put(PKIAuthenticationPlugin.PATH, pkiAuthenticationPlugin.getRequestHandler());
 
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.node);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jvm);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.jetty);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, SolrInfoMBean.Group.http);
+
     coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
 
     containerProperties.putAll(cfg.getSolrProperties());
 
+    // initialize gauges for reporting the number of cores
+    Gauge<Integer> loadedCores = () -> solrCores.getCores().size();
+    Gauge<Integer> lazyCores = () -> solrCores.getCoreNames().size() - solrCores.getCores().size();
+    Gauge<Integer> unloadedCores = () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size();
+
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        loadedCores, true, "loaded", "cores");
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        lazyCores, true, "lazy", "cores");
+    metricManager.register(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node),
+        unloadedCores, true, "unloaded", "cores");
+
     // setup executor to load cores in parallel
     ExecutorService coreLoadExecutor = ExecutorUtil.newMDCAwareFixedThreadPool(
         cfg.getCoreLoadThreadCount(isZooKeeperAware()),
@@ -658,6 +688,10 @@ public class CoreContainer {
       }
     }
 
+    if (metricManager != null) {
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+    }
+
     // It should be safe to close the authorization plugin at this point.
     try {
       if(authorizationPlugin != null) {
@@ -1034,6 +1068,9 @@ public class CoreContainer {
     SolrCore core = solrCores.remove(name);
     coresLocator.delete(this, cd);
 
+    // delete metrics specific to this core
+    metricManager.removeRegistry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, name));
+
     if (core == null) {
       // transient core
       SolrCore.deleteUnloadedCore(cd, deleteDataDir, deleteInstanceDir);
@@ -1167,8 +1204,15 @@ public class CoreContainer {
 
   // ---------------- CoreContainer request handlers --------------
 
-  protected <T> T createHandler(String handlerClass, Class<T> clazz) {
-    return loader.newInstance(handlerClass, clazz, null, new Class[] { CoreContainer.class }, new Object[] { this });
+  protected <T> T createHandler(String path, String handlerClass, Class<T> clazz) {
+    T handler = loader.newInstance(handlerClass, clazz, null, new Class[] { CoreContainer.class }, new Object[] { this });
+    if (handler instanceof SolrRequestHandler) {
+      containerHandlers.put(path, (SolrRequestHandler)handler);
+    }
+    if (handler instanceof SolrMetricProducer) {
+      ((SolrMetricProducer)handler).initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), path);
+    }
+    return handler;
   }
 
   public CoreAdminHandler getMultiCoreHandler() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/NodeConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index ea451a9..258fd14 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -60,13 +60,16 @@ public class NodeConfig {
 
   private final PluginInfo[] backupRepositoryPlugins;
 
+  private final PluginInfo[] metricReporterPlugins;
+
   private NodeConfig(String nodeName, Path coreRootDirectory, Path configSetBaseDirectory, String sharedLibDirectory,
                      PluginInfo shardHandlerFactoryConfig, UpdateShardHandlerConfig updateShardHandlerConfig,
                      String coreAdminHandlerClass, String collectionsAdminHandlerClass,
                      String infoHandlerClass, String configSetsHandlerClass,
                      LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, Integer coreLoadThreads,
                      int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader,
-                     Properties solrProperties, PluginInfo[] backupRepositoryPlugins) {
+                     Properties solrProperties, PluginInfo[] backupRepositoryPlugins,
+                     PluginInfo[] metricReporterPlugins) {
     this.nodeName = nodeName;
     this.coreRootDirectory = coreRootDirectory;
     this.configSetBaseDirectory = configSetBaseDirectory;
@@ -86,6 +89,7 @@ public class NodeConfig {
     this.loader = loader;
     this.solrProperties = solrProperties;
     this.backupRepositoryPlugins = backupRepositoryPlugins;
+    this.metricReporterPlugins = metricReporterPlugins;
 
     if (this.cloudConfig != null && this.getCoreLoadThreadCount(false) < 2) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -174,6 +178,10 @@ public class NodeConfig {
     return backupRepositoryPlugins;
   }
 
+  public PluginInfo[] getMetricReporterPlugins() {
+    return metricReporterPlugins;
+  }
+
   public static class NodeConfigBuilder {
 
     private Path coreRootDirectory;
@@ -193,6 +201,7 @@ public class NodeConfig {
     private String managementPath;
     private Properties solrProperties = new Properties();
     private PluginInfo[] backupRepositoryPlugins;
+    private PluginInfo[] metricReporterPlugins;
 
     private final SolrResourceLoader loader;
     private final String nodeName;
@@ -300,11 +309,16 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setMetricReporterPlugins(PluginInfo[] metricReporterPlugins) {
+      this.metricReporterPlugins = metricReporterPlugins;
+      return this;
+    }
+
     public NodeConfig build() {
       return new NodeConfig(nodeName, coreRootDirectory, configSetBaseDirectory, sharedLibDirectory, shardHandlerFactoryConfig,
                             updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, infoHandlerClass, configSetsHandlerClass,
                             logWatcherConfig, cloudConfig, coreLoadThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties,
-                            backupRepositoryPlugins);
+                            backupRepositoryPlugins, metricReporterPlugins);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/PluginInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginInfo.java b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
index 5fdef3c..10f8b8d 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginInfo.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginInfo.java
@@ -111,6 +111,7 @@ public class PluginInfo implements MapSerializable {
     if (type != null) sb.append("type = " + type + ",");
     if (name != null) sb.append("name = " + name + ",");
     if (className != null) sb.append("class = " + className + ",");
+    if (attributes != null && attributes.size() > 0) sb.append("attributes = " + attributes + ",");
     if (initArgs != null && initArgs.size() > 0) sb.append("args = " + initArgs);
     sb.append("}");
     return sb.toString();
@@ -181,7 +182,8 @@ public class PluginInfo implements MapSerializable {
 
   }
   public PluginInfo copy() {
-    PluginInfo result = new PluginInfo(type, attributes, initArgs.clone(), children);
+    PluginInfo result = new PluginInfo(type, attributes,
+        initArgs != null ? initArgs.clone() : null, children);
     result.isFromSolrConfig = isFromSolrConfig;
     return result;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index f75e780..a8d7738 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -53,6 +53,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Timer;
 import com.google.common.collect.MapMaker;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.analysis.util.ResourceLoader;
@@ -94,6 +96,9 @@ import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.metrics.SolrCoreMetricManager;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.BinaryResponseWriter;
@@ -187,6 +192,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final PluginBag<SearchComponent> searchComponents = new PluginBag<>(SearchComponent.class, this);
   private final PluginBag<UpdateRequestProcessorFactory> updateProcessors = new PluginBag<>(UpdateRequestProcessorFactory.class, this, true);
   private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
+  private final SolrCoreMetricManager coreMetricManager;
   private final Map<String, SolrInfoMBean> infoRegistry;
   private final IndexDeletionPolicyWrapper solrDelPolicy;
   private final SolrSnapshotMetaDataManager snapshotMgr;
@@ -200,6 +206,12 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private final ReentrantLock ruleExpiryLock;
   private final ReentrantLock snapshotDelLock; // A lock instance to guard against concurrent deletions.
 
+  private final Timer newSearcherTimer;
+  private final Timer newSearcherWarmupTimer;
+  private final Counter newSearcherCounter;
+  private final Counter newSearcherMaxReachedCounter;
+  private final Counter newSearcherOtherErrorsCounter;
+
   public Date getStartTimeStamp() { return startTime; }
 
   private final Map<Object, IndexFingerprint> perSegmentFingerprintCache = new MapMaker().weakKeys().makeMap();
@@ -386,9 +398,13 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   public void setName(String v) {
+    String oldName = this.name;
     this.name = v;
     this.logid = (v==null)?"":("["+v+"] ");
     this.coreDescriptor = new CoreDescriptor(v, this.coreDescriptor);
+    if (coreMetricManager != null) {
+      coreMetricManager.afterCoreSetName();
+    }
   }
 
   public String getLogId()
@@ -397,6 +413,15 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   /**
+   * Returns the {@link SolrCoreMetricManager} for this core.
+   *
+   * @return the {@link SolrCoreMetricManager} for this core
+   */
+  public SolrCoreMetricManager getCoreMetricManager() {
+    return coreMetricManager;
+  }
+
+  /**
    * Returns a Map of name vs SolrInfoMBean objects. The returned map is an instance of
    * a ConcurrentHashMap and therefore no synchronization is needed for putting, removing
    * or iterating over it.
@@ -838,6 +863,18 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
     checkVersionFieldExistsInSchema(schema, coreDescriptor);
 
+    // Initialize the metrics manager
+    this.coreMetricManager = initCoreMetricManager(config);
+
+    SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager();
+
+    // initialize searcher-related metrics
+    newSearcherCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcher");
+    newSearcherTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherTime");
+    newSearcherWarmupTimer = metricManager.timer(coreMetricManager.getRegistryName(), "newSearcherWarmup");
+    newSearcherMaxReachedCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherMaxReached");
+    newSearcherOtherErrorsCounter = metricManager.counter(coreMetricManager.getRegistryName(), "newSearcherErrors");
+
     // Initialize JMX
     this.infoRegistry = initInfoRegistry(name, config);
     infoRegistry.put("fieldCache", new SolrFieldCacheMBean());
@@ -1041,6 +1078,19 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     setLatestSchema(schema);
   }
 
+  /**
+   * Initializes the core's {@link SolrCoreMetricManager} with a given configuration.
+   * If metric reporters are configured, they are also initialized for this core.
+   *
+   * @param config the given configuration
+   * @return an instance of {@link SolrCoreMetricManager}
+   */
+  private SolrCoreMetricManager initCoreMetricManager(SolrConfig config) {
+    SolrCoreMetricManager coreMetricManager = new SolrCoreMetricManager(this);
+    coreMetricManager.loadReporters();
+    return coreMetricManager;
+  }
+
   private Map<String,SolrInfoMBean> initInfoRegistry(String name, SolrConfig config) {
     if (config.jmxConfig.enabled) {
       return new JmxMonitoredMap<String, SolrInfoMBean>(name, String.valueOf(this.hashCode()), config.jmxConfig);
@@ -1361,6 +1411,15 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       }
     }
 
+    try {
+      coreMetricManager.close();
+    } catch (Throwable e) {
+      SolrException.log(log, e);
+      if (e instanceof  Error) {
+        throw (Error) e;
+      }
+    }
+
     // Close the snapshots meta-data directory.
     Directory snapshotsDir = snapshotMgr.getSnapshotsDir();
     try {
@@ -1920,12 +1979,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
         // first: increment count to signal other threads that we are
         //        opening a new searcher.
         onDeckSearchers++;
+        newSearcherCounter.inc();
         if (onDeckSearchers < 1) {
           // should never happen... just a sanity check
           log.error(logid + "ERROR!!! onDeckSearchers is " + onDeckSearchers);
           onDeckSearchers = 1;  // reset
         } else if (onDeckSearchers > maxWarmingSearchers) {
           onDeckSearchers--;
+          newSearcherMaxReachedCounter.inc();
           try {
             searcherLock.wait();
           } catch (InterruptedException e) {
@@ -1947,6 +2008,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     boolean success = false;
 
     openSearcherLock.lock();
+    Timer.Context timerContext = newSearcherTimer.time();
     try {
       searchHolder = openNewSearcher(updateHandlerReopens, false);
        // the searchHolder will be incremented once already (and it will eventually be assigned to _searcher when registered)
@@ -1989,6 +2051,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
         // should this go before the other event handlers or after?
         if (currSearcher != null) {
           future = searcherExecutor.submit(() -> {
+            Timer.Context warmupContext = newSearcherWarmupTimer.time();
             try {
               newSearcher.warm(currSearcher);
             } catch (Throwable e) {
@@ -1996,6 +2059,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
               if (e instanceof Error) {
                 throw (Error) e;
               }
+            } finally {
+              warmupContext.close();
             }
             return null;
           });
@@ -2076,7 +2141,10 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     } finally {
 
+      timerContext.close();
+
       if (!success) {
+        newSearcherOtherErrorsCounter.inc();;
         synchronized (searcherLock) {
           onDeckSearchers--;
 
@@ -2750,6 +2818,11 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
   public void registerInfoBean(String name, SolrInfoMBean solrInfoMBean) {
     infoRegistry.put(name, solrInfoMBean);
+
+    if (solrInfoMBean instanceof SolrMetricProducer) {
+      SolrMetricProducer producer = (SolrMetricProducer) solrInfoMBean;
+      coreMetricManager.registerMetricProducer(name, producer);
+    }
   }
 
   private static boolean checkStale(SolrZkClient zkClient,  String zkPath, int currentVersion)  {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
index c119e2f..c5fb84b 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrInfoMBean.java
@@ -29,7 +29,15 @@ import org.apache.solr.common.util.NamedList;
  */
 public interface SolrInfoMBean {
 
-  public enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER };
+  /**
+   * Category of {@link SolrCore} component.
+   */
+  enum Category { CORE, QUERYHANDLER, UPDATEHANDLER, CACHE, HIGHLIGHTING, QUERYPARSER, OTHER }
+
+  /**
+   * Top-level group of beans for a subsystem.
+   */
+  enum Group { jvm, jetty, http, node, core }
 
   /**
    * Simple common usage name, e.g. BasicQueryHandler,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 65b248d..49d9ae5 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -95,7 +95,8 @@ public class SolrXmlConfig {
     configBuilder.setSolrProperties(loadProperties(config));
     if (cloudConfig != null)
       configBuilder.setCloudConfig(cloudConfig);
-    configBuilder.setBackupRepositoryPlugins((getBackupRepositoryPluginInfos(config)));
+    configBuilder.setBackupRepositoryPlugins(getBackupRepositoryPluginInfos(config));
+    configBuilder.setMetricReporterPlugins(getMetricReporterPluginInfos(config));
     return fillSolrSection(configBuilder, entries);
   }
 
@@ -436,5 +437,16 @@ public class SolrXmlConfig {
     }
     return configs;
   }
+
+  private static PluginInfo[] getMetricReporterPluginInfos(Config config) {
+    NodeList nodes = (NodeList) config.evaluate("solr/metrics/reporter", XPathConstants.NODESET);
+    if (nodes == null || nodes.getLength() == 0)
+      return new PluginInfo[0];
+    PluginInfo[] configs = new PluginInfo[nodes.getLength()];
+    for (int i = 0; i < nodes.getLength(); i++) {
+      configs[i] = new PluginInfo(nodes.item(i), "SolrMetricReporter", true, true);
+    }
+    return configs;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
index 218b6de..85597dc 100644
--- a/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/RequestHandlerBase.java
@@ -18,8 +18,11 @@ package org.apache.solr.handler;
 
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
-import java.util.concurrent.atomic.LongAdder;
+import java.util.Arrays;
+import java.util.Collection;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Meter;
 import com.codahale.metrics.Timer;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
@@ -29,12 +32,14 @@ import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.SolrPluginUtils;
-import org.apache.solr.util.stats.TimerUtils;
+import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,7 +48,7 @@ import static org.apache.solr.core.RequestParams.USEPARAM;
 /**
  *
  */
-public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, NestedRequestHandler {
+public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfoMBean, SolrMetricProducer, NestedRequestHandler {
 
   protected NamedList initArgs = null;
   protected SolrParams defaults;
@@ -52,11 +57,12 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   protected boolean httpCaching = true;
 
   // Statistics
-  private final LongAdder numRequests = new LongAdder();
-  private final LongAdder numServerErrors = new LongAdder();
-  private final LongAdder numClientErrors = new LongAdder();
-  private final LongAdder numTimeouts = new LongAdder();
-  private final Timer requestTimes = new Timer();
+  private Meter numErrors = new Meter();
+  private Meter numServerErrors = new Meter();
+  private Meter numClientErrors = new Meter();
+  private Meter numTimeouts = new Meter();
+  private Counter requests = new Counter();
+  private Timer requestTimes = new Timer();
 
   private final long handlerStart;
 
@@ -126,6 +132,17 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
 
   }
 
+  @Override
+  public Collection<String> initializeMetrics(SolrMetricManager manager, String registryName, String scope) {
+    numErrors = manager.meter(registryName, "errors", getCategory().toString(), scope);
+    numServerErrors = manager.meter(registryName, "serverErrors", getCategory().toString(), scope);
+    numClientErrors = manager.meter(registryName, "clientErrors", getCategory().toString(), scope);
+    numTimeouts = manager.meter(registryName, "timeouts", getCategory().toString(), scope);
+    requests = manager.counter(registryName, "requests", getCategory().toString(), scope);
+    requestTimes = manager.timer(registryName, "requestTimes", getCategory().toString(), scope);
+    return Arrays.asList("errors", "serverErrors", "clientErrors", "timeouts", "requestTimes", "requests");
+  }
+
   public static SolrParams getSolrParamsFromNamedList(NamedList args, String key) {
     Object o = args.get(key);
     if (o != null && o instanceof NamedList) {
@@ -142,7 +159,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
 
   @Override
   public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
-    numRequests.increment();
+    requests.inc();
     Timer.Context timer = requestTimes.time();
     try {
       if(pluginInfo != null && pluginInfo.attributes.containsKey(USEPARAM)) req.getContext().put(USEPARAM,pluginInfo.attributes.get(USEPARAM));
@@ -156,7 +173,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
         Object partialResults = header.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY);
         boolean timedOut = partialResults == null ? false : (Boolean)partialResults;
         if( timedOut ) {
-          numTimeouts.increment();
+          numTimeouts.mark();
           rsp.setHttpCaching(false);
         }
       }
@@ -182,14 +199,14 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
       if (incrementErrors) {
         SolrException.log(log, e);
 
+        numErrors.mark();
         if (isServerError) {
-          numServerErrors.increment();
+          numServerErrors.mark();
         } else {
-          numClientErrors.increment();
+          numClientErrors.mark();
         }
       }
-    }
-    finally {
+    } finally {
       timer.stop();
     }
   }
@@ -268,15 +285,14 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
   public NamedList<Object> getStatistics() {
     NamedList<Object> lst = new SimpleOrderedMap<>();
     lst.add("handlerStart",handlerStart);
-    lst.add("requests", numRequests.longValue());
-    lst.add("errors", numServerErrors.longValue() + numClientErrors.longValue());
-    lst.add("serverErrors", numServerErrors.longValue());
-    lst.add("clientErrors", numClientErrors.longValue());
-    lst.add("timeouts", numTimeouts.longValue());
-    TimerUtils.addMetrics(lst, requestTimes);
+    lst.add("requests", requests.getCount());
+    lst.add("errors", numErrors.getCount());
+    lst.add("serverErrors", numServerErrors.getCount());
+    lst.add("clientErrors", numClientErrors.getCount());
+    lst.add("timeouts", numTimeouts.getCount());
+    MetricUtils.addMetrics(lst, requestTimes);
     return lst;
   }
-
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
new file mode 100644
index 0000000..78b2045
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java
@@ -0,0 +1,164 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
+import org.apache.solr.util.stats.MetricUtils;
+
+/**
+ * Request handler to return metrics
+ */
+public class MetricsHandler extends RequestHandlerBase implements PermissionNameProvider {
+  final CoreContainer container;
+  final SolrMetricManager metricManager;
+
+  public MetricsHandler() {
+    this.container = null;
+    this.metricManager = null;
+  }
+
+  public MetricsHandler(CoreContainer container) {
+    this.container = container;
+    this.metricManager = this.container.getMetricManager();
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext request) {
+    return Name.METRICS_READ_PERM;
+  }
+
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    if (container == null) {
+      throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Core container instance not initialized");
+    }
+
+    List<MetricType> metricTypes = parseMetricTypes(req);
+    List<MetricFilter> metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList());
+    List<Group> requestedGroups = parseGroups(req);
+
+    NamedList response = new NamedList();
+    for (Group group : requestedGroups) {
+      String registryName = SolrMetricManager.getRegistryName(group);
+      if (group == Group.core) {
+        // this requires special handling because of the way we create registry name for a core (deeply nested)
+        container.getAllCoreNames().forEach(s -> {
+          String coreRegistryName;
+          try (SolrCore core = container.getCore(s)) {
+            coreRegistryName = core.getCoreMetricManager().getRegistryName();
+          }
+          MetricRegistry registry = metricManager.registry(coreRegistryName);
+          response.add(coreRegistryName, MetricUtils.toNamedList(registry, metricFilters));
+        });
+      } else {
+        MetricRegistry registry = metricManager.registry(registryName);
+        response.add(registryName, MetricUtils.toNamedList(registry, metricFilters));
+      }
+    }
+    rsp.getValues().add("metrics", response);
+  }
+
+  private List<Group> parseGroups(SolrQueryRequest req) {
+    String[] groupStr = req.getParams().getParams("group");
+    List<String> groups = Collections.emptyList();
+    if (groupStr != null && groupStr.length > 0) {
+      groups = new ArrayList<>();
+      for (String g : groupStr) {
+        groups.addAll(StrUtils.splitSmart(g, ','));
+      }
+    }
+
+    List<Group> requestedGroups = Arrays.asList(Group.values()); // by default we return all groups
+    try {
+      if (groups.size() > 0 && !groups.contains("all")) {
+        requestedGroups = groups.stream().map(String::trim).map(Group::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid group in: " + groups + " specified. Must be one of (all, jvm, jetty, http, node, core)", e);
+    }
+    return requestedGroups;
+  }
+
+  private List<MetricType> parseMetricTypes(SolrQueryRequest req) {
+    String[] typeStr = req.getParams().getParams("type");
+    List<String> types = Collections.emptyList();
+    if (typeStr != null && typeStr.length > 0)  {
+      types = new ArrayList<>();
+      for (String type : typeStr) {
+        types.addAll(StrUtils.splitSmart(type, ','));
+      }
+    }
+
+    List<MetricType> metricTypes = Collections.singletonList(MetricType.all); // include all metrics by default
+    try {
+      if (types.size() > 0) {
+        metricTypes = types.stream().map(String::trim).map(MetricType::valueOf).collect(Collectors.toList());
+      }
+    } catch (IllegalArgumentException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid metric type in: " + types + " specified. Must be one of (all, meter, timer, histogram, counter, gauge)", e);
+    }
+    return metricTypes;
+  }
+
+  @Override
+  public String getDescription() {
+    return "A handler to return all the metrics gathered by Solr";
+  }
+
+  enum MetricType {
+    histogram(Histogram.class),
+    meter(Meter.class),
+    timer(Timer.class),
+    counter(Counter.class),
+    gauge(Gauge.class),
+    all(null);
+
+    private final Class klass;
+
+    MetricType(Class klass) {
+      this.klass = klass;
+    }
+
+    public MetricFilter asMetricFilter() {
+      return (name, metric) -> klass == null || klass.isInstance(metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
new file mode 100644
index 0000000..0e5403b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrCoreMetricManager.java
@@ -0,0 +1,154 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Collection;
+import java.lang.invoke.MethodHandles;
+
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoMBean;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Responsible for collecting metrics from {@link SolrMetricProducer}'s
+ * and exposing metrics to {@link SolrMetricReporter}'s.
+ */
+public class SolrCoreMetricManager implements Closeable {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final SolrCore core;
+  private final SolrMetricManager metricManager;
+  private String registryName;
+
+  /**
+   * Constructs a metric manager.
+   *
+   * @param core the metric manager's core
+   */
+  public SolrCoreMetricManager(SolrCore core) {
+    this.core = core;
+    this.metricManager = core.getCoreDescriptor().getCoreContainer().getMetricManager();
+    registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
+  }
+
+  /**
+   * Load reporters configured globally and specific to {@link org.apache.solr.core.SolrInfoMBean.Group#core}
+   * group or with a registry name specific to this core.
+   */
+  public void loadReporters() {
+    NodeConfig nodeConfig = core.getCoreDescriptor().getCoreContainer().getConfig();
+    PluginInfo[] pluginInfos = nodeConfig.getMetricReporterPlugins();
+    metricManager.loadReporters(pluginInfos, core.getResourceLoader(), SolrInfoMBean.Group.core, registryName);
+  }
+
+  /**
+   * Make sure that metrics already collected that correspond to the old core name
+   * are carried over and will be used under the new core name.
+   * This method also reloads reporters so that they use the new core name.
+   */
+  public void afterCoreSetName() {
+    String oldRegistryName = registryName;
+    registryName = createRegistryName(core.getCoreDescriptor().getCollectionName(), core.getName());
+    if (oldRegistryName.equals(registryName)) {
+      return;
+    }
+    // close old reporters
+    metricManager.closeReporters(oldRegistryName);
+    metricManager.moveMetrics(oldRegistryName, registryName, null);
+    // old registry is no longer used - we have moved the metrics
+    metricManager.removeRegistry(oldRegistryName);
+    // load reporters again, using the new core name
+    loadReporters();
+  }
+
+  /**
+   * Registers a mapping of name/metric's with the manager's metric registry.
+   *
+   * @param scope     the scope of the metrics to be registered (e.g. `/admin/ping`)
+   * @param producer  producer of metrics to be registered
+   */
+  public void registerMetricProducer(String scope, SolrMetricProducer producer) {
+    if (scope == null || producer == null || producer.getCategory() == null) {
+      throw new IllegalArgumentException("registerMetricProducer() called with illegal arguments: " +
+          "scope = " + scope + ", producer = " + producer);
+    }
+    Collection<String> registered = producer.initializeMetrics(metricManager, getRegistryName(), scope);
+    if (registered == null || registered.isEmpty()) {
+      throw new IllegalArgumentException("registerMetricProducer() did not register any metrics " +
+      "for scope = " + scope + ", producer = " + producer);
+    }
+  }
+
+  /**
+   * Closes reporters specific to this core.
+   */
+  @Override
+  public void close() throws IOException {
+    metricManager.closeReporters(getRegistryName());
+  }
+
+  public SolrCore getCore() {
+    return core;
+  }
+
+  /**
+   * Retrieves the metric registry name of the manager.
+   *
+   * In order to make it easier for reporting tools to aggregate metrics from
+   * different cores that logically belong to a single collection we convert the
+   * core name into a dot-separated hierarchy of: collection name, shard name (with optional split)
+   * and replica name.
+   *
+   * <p>For example, when the core name looks like this but it's NOT a SolrCloud collection:
+   * <code>my_collection_shard1_1_replica1</code> then this will be used as the registry name (plus
+   * the required <code>solr.core</code> prefix). However,
+   * if this is a SolrCloud collection <code>my_collection</code> then the registry name will become
+   * <code>solr.core.my_collection.shard1_1.replica1</code>.</p>
+   *
+   *
+   * @return the metric registry name of the manager.
+   */
+  public String getRegistryName() {
+    return registryName;
+  }
+
+  /* package visibility for tests. */
+  String createRegistryName(String collectionName, String coreName) {
+    if (collectionName == null || (collectionName != null && !coreName.startsWith(collectionName + "_"))) {
+      // single core, or unknown naming scheme
+      return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, coreName);
+    }
+    // split "collection1_shard1_1_replica1" into parts
+    String str = coreName.substring(collectionName.length() + 1);
+    String shard;
+    String replica = null;
+    int pos = str.lastIndexOf("_replica");
+    if (pos == -1) { // ?? no _replicaN part ??
+      shard = str;
+    } else {
+      shard = str.substring(0, pos);
+      replica = str.substring(pos + 1);
+    }
+    return SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, collectionName, shard, replica);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java b/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
new file mode 100644
index 0000000..f0bc8a1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricInfo.java
@@ -0,0 +1,104 @@
+/*
+ * 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 com.codahale.metrics.MetricRegistry;
+import org.apache.solr.core.SolrInfoMBean;
+
+/**
+ * Wraps meta-data for a metric.
+ */
+public final class SolrMetricInfo {
+  public final String name;
+  public final String scope;
+  public final SolrInfoMBean.Category category;
+
+  /**
+   * Creates a new instance of {@link SolrMetricInfo}.
+   *
+   * @param category the category of the metric (e.g. `QUERYHANDLERS`)
+   * @param scope    the scope of the metric (e.g. `/admin/ping`)
+   * @param name     the name of the metric (e.g. `Requests`)
+   */
+  public SolrMetricInfo(SolrInfoMBean.Category category, String scope, String name) {
+    this.name = name;
+    this.scope = scope;
+    this.category = category;
+  }
+
+  public static SolrMetricInfo of(String fullName) {
+    if (fullName == null || fullName.isEmpty()) {
+      return null;
+    }
+    String[] names = fullName.split("\\.");
+    if (names.length < 3) { // not a valid info
+      return null;
+    }
+    // check top-level name for valid category
+    SolrInfoMBean.Category category;
+    try {
+      category = SolrInfoMBean.Category.valueOf(names[0]);
+    } catch (IllegalArgumentException e) { // not a valid category
+      return null;
+    }
+    String scope = names[1];
+    String name = fullName.substring(names[0].length() + names[1].length() + 2);
+    return new SolrMetricInfo(category, scope, name);
+  }
+
+  /**
+   * Returns the metric name defined by this object.
+   * For example, if the name is `Requests`, scope is `/admin/ping`,
+   * and category is `QUERYHANDLERS`, then the metric name is
+   * `QUERYHANDLERS./admin/ping.Requests`.
+   *
+   * @return the metric name defined by this object
+   */
+  public String getMetricName() {
+    return MetricRegistry.name(category.toString(), scope, name);
+  }
+
+  @Override
+  public String toString() {
+    return "SolrMetricInfo{" +
+        "name='" + name + '\'' +
+        ", scope='" + scope + '\'' +
+        ", category=" + category +
+        '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    SolrMetricInfo that = (SolrMetricInfo) o;
+
+    if (name != null ? !name.equals(that.name) : that.name != null) return false;
+    if (scope != null ? !scope.equals(that.scope) : that.scope != null) return false;
+    return category == that.category;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = name != null ? name.hashCode() : 0;
+    result = 31 * result + (scope != null ? scope.hashCode() : 0);
+    result = 31 * result + (category != null ? category.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8bbdb624/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..3f51aef
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/metrics/SolrMetricManager.java
@@ -0,0 +1,652 @@
+/*
+ * 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());
+
+  public static final String REGISTRY_NAME_PREFIX = "solr.";
+
+  public static final String JETTY_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jetty.toString();
+
+  public static final String JVM_REGISTRY = REGISTRY_NAME_PREFIX + SolrInfoMBean.Group.jvm.toString();
+
+  private final ConcurrentMap<String, MetricRegistry> registries = new ConcurrentHashMap<>();
+
+  // these reporters are per CoreContainer
+  private final Map<String, Map<String, SolrMetricReporter>> reporters = new HashMap<>();
+
+  // these reporters are per JVM
+  private static final Map<String, Map<String, SolrMetricReporter>> sharedReporters = 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);
+  }
+
+  /**
+   * 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 (JETTY_REGISTRY.equals(registry) || JVM_REGISTRY.equals(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 (JETTY_REGISTRY.equals(registry) || JVM_REGISTRY.equals(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/8bbdb624/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);
+}